[jira] [Commented] (FLINK-16589) Flink Table SQL fails/crashes with big queries with lots of fields
[ https://issues.apache.org/jira/browse/FLINK-16589?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17140331#comment-17140331 ] Benchao Li commented on FLINK-16589: Fixed via * release-1.11: 334b7b8b00885135b0682756f970b4e440f0f189 * master: 99fca58fe60199b93f962e5fbf0cb0314b9d3b99 > Flink Table SQL fails/crashes with big queries with lots of fields > -- > > Key: FLINK-16589 > URL: https://issues.apache.org/jira/browse/FLINK-16589 > Project: Flink > Issue Type: Bug > Components: Table SQL / Planner >Affects Versions: 1.10.0 >Reporter: Viet Pham >Assignee: Benchao Li >Priority: Critical > Labels: pull-request-available > Fix For: 1.11.0 > > Time Spent: 10m > Remaining Estimate: 0h > > Hi, > My use case is a streaming application with a few streaming tables. > I was trying to build a SELECT query (and registering it as a temporary view) > with about 200 fields/expressions out of another streaming table. The > application is successfully submitted to Flink cluster. However the worker > processes keep crashing, with the exception as quoted below. > It clearly mentioned in the log that this is a bug, so I fire this ticket. By > the way, if I lower the number of fields down to 100 then it works nicely. > Please advice. > Thanks a lot for all the efforts bring Flink up. It is really amazing! > {code:java} > java.lang.RuntimeException: Could not instantiate generated class > 'GroupAggsHandler$9687'at > org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:57) > at > org.apache.flink.table.runtime.operators.aggregate.MiniBatchGroupAggFunction.open(MiniBatchGroupAggFunction.java:136) > at > org.apache.flink.table.runtime.operators.bundle.AbstractMapBundleOperator.open(AbstractMapBundleOperator.java:84) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.initializeStateAndOpen(StreamTask.java:1007) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$beforeInvoke$0(StreamTask.java:454) > at > org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.runThrowing(StreamTaskActionExecutor.java:94) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.beforeInvoke(StreamTask.java:449) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:461) > at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:707)at > org.apache.flink.runtime.taskmanager.Task.run(Task.java:532)at > java.lang.Thread.run(Thread.java:748)Caused by: > org.apache.flink.util.FlinkRuntimeException: > org.apache.flink.api.common.InvalidProgramException: Table program cannot be > compiled. This is a bug. Please file an issue.at > org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:68) > at > org.apache.flink.table.runtime.generated.GeneratedClass.compile(GeneratedClass.java:78) > at > org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:52) > ... 10 moreCaused by: > org.apache.flink.shaded.guava18.com.google.common.util.concurrent.UncheckedExecutionException: > org.apache.flink.api.common.InvalidProgramException: Table program cannot be > compiled. This is a bug. Please file an issue.at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2203) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache.get(LocalCache.java:3937) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4739) > at > org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:66) > ... 12 moreCaused by: > org.apache.flink.api.common.InvalidProgramException: Table program cannot be > compiled. This is a bug. Please file an issue.at > org.apache.flink.table.runtime.generated.CompileUtils.doCompile(CompileUtils.java:81) > at > org.apache.flink.table.runtime.generated.CompileUtils.lambda$compile$1(CompileUtils.java:66) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalManualCache$1.load(LocalCache.java:4742) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3527) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2319) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2282) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2197) > ... 15 moreCaused by:
[jira] [Commented] (FLINK-16589) Flink Table SQL fails/crashes with big queries with lots of fields
[ https://issues.apache.org/jira/browse/FLINK-16589?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17123490#comment-17123490 ] Danny Chen commented on FLINK-16589: cc [~jark] > Flink Table SQL fails/crashes with big queries with lots of fields > -- > > Key: FLINK-16589 > URL: https://issues.apache.org/jira/browse/FLINK-16589 > Project: Flink > Issue Type: Bug > Components: Table SQL / Planner >Affects Versions: 1.10.0 >Reporter: Viet Pham >Assignee: Benchao Li >Priority: Major > Labels: pull-request-available > Fix For: 1.11.0 > > Time Spent: 10m > Remaining Estimate: 0h > > Hi, > My use case is a streaming application with a few streaming tables. > I was trying to build a SELECT query (and registering it as a temporary view) > with about 200 fields/expressions out of another streaming table. The > application is successfully submitted to Flink cluster. However the worker > processes keep crashing, with the exception as quoted below. > It clearly mentioned in the log that this is a bug, so I fire this ticket. By > the way, if I lower the number of fields down to 100 then it works nicely. > Please advice. > Thanks a lot for all the efforts bring Flink up. It is really amazing! > {code:java} > java.lang.RuntimeException: Could not instantiate generated class > 'GroupAggsHandler$9687'at > org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:57) > at > org.apache.flink.table.runtime.operators.aggregate.MiniBatchGroupAggFunction.open(MiniBatchGroupAggFunction.java:136) > at > org.apache.flink.table.runtime.operators.bundle.AbstractMapBundleOperator.open(AbstractMapBundleOperator.java:84) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.initializeStateAndOpen(StreamTask.java:1007) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$beforeInvoke$0(StreamTask.java:454) > at > org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.runThrowing(StreamTaskActionExecutor.java:94) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.beforeInvoke(StreamTask.java:449) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:461) > at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:707)at > org.apache.flink.runtime.taskmanager.Task.run(Task.java:532)at > java.lang.Thread.run(Thread.java:748)Caused by: > org.apache.flink.util.FlinkRuntimeException: > org.apache.flink.api.common.InvalidProgramException: Table program cannot be > compiled. This is a bug. Please file an issue.at > org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:68) > at > org.apache.flink.table.runtime.generated.GeneratedClass.compile(GeneratedClass.java:78) > at > org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:52) > ... 10 moreCaused by: > org.apache.flink.shaded.guava18.com.google.common.util.concurrent.UncheckedExecutionException: > org.apache.flink.api.common.InvalidProgramException: Table program cannot be > compiled. This is a bug. Please file an issue.at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2203) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache.get(LocalCache.java:3937) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4739) > at > org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:66) > ... 12 moreCaused by: > org.apache.flink.api.common.InvalidProgramException: Table program cannot be > compiled. This is a bug. Please file an issue.at > org.apache.flink.table.runtime.generated.CompileUtils.doCompile(CompileUtils.java:81) > at > org.apache.flink.table.runtime.generated.CompileUtils.lambda$compile$1(CompileUtils.java:66) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalManualCache$1.load(LocalCache.java:4742) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3527) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2319) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2282) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2197) > ... 15 moreCaused by: org.codehaus.janino.InternalCompilerException: > Compiling "GroupAggsHandler$9687": Code of method >
[jira] [Commented] (FLINK-16589) Flink Table SQL fails/crashes with big queries with lots of fields
[ https://issues.apache.org/jira/browse/FLINK-16589?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17059536#comment-17059536 ] Benchao Li commented on FLINK-16589: [~StarGhost] Thanks for the example. I'll fix this in a few days. > Flink Table SQL fails/crashes with big queries with lots of fields > -- > > Key: FLINK-16589 > URL: https://issues.apache.org/jira/browse/FLINK-16589 > Project: Flink > Issue Type: Bug > Components: Table SQL / API >Affects Versions: 1.10.0 >Reporter: Viet Pham >Assignee: Benchao Li >Priority: Critical > Fix For: 1.10.1, 1.11.0 > > > Hi, > My use case is a streaming application with a few streaming tables. > I was trying to build a SELECT query (and registering it as a temporary view) > with about 200 fields/expressions out of another streaming table. The > application is successfully submitted to Flink cluster. However the worker > processes keep crashing, with the exception as quoted below. > It clearly mentioned in the log that this is a bug, so I fire this ticket. By > the way, if I lower the number of fields down to 100 then it works nicely. > Please advice. > Thanks a lot for all the efforts bring Flink up. It is really amazing! > {code:java} > java.lang.RuntimeException: Could not instantiate generated class > 'GroupAggsHandler$9687'at > org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:57) > at > org.apache.flink.table.runtime.operators.aggregate.MiniBatchGroupAggFunction.open(MiniBatchGroupAggFunction.java:136) > at > org.apache.flink.table.runtime.operators.bundle.AbstractMapBundleOperator.open(AbstractMapBundleOperator.java:84) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.initializeStateAndOpen(StreamTask.java:1007) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$beforeInvoke$0(StreamTask.java:454) > at > org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.runThrowing(StreamTaskActionExecutor.java:94) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.beforeInvoke(StreamTask.java:449) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:461) > at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:707)at > org.apache.flink.runtime.taskmanager.Task.run(Task.java:532)at > java.lang.Thread.run(Thread.java:748)Caused by: > org.apache.flink.util.FlinkRuntimeException: > org.apache.flink.api.common.InvalidProgramException: Table program cannot be > compiled. This is a bug. Please file an issue.at > org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:68) > at > org.apache.flink.table.runtime.generated.GeneratedClass.compile(GeneratedClass.java:78) > at > org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:52) > ... 10 moreCaused by: > org.apache.flink.shaded.guava18.com.google.common.util.concurrent.UncheckedExecutionException: > org.apache.flink.api.common.InvalidProgramException: Table program cannot be > compiled. This is a bug. Please file an issue.at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2203) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache.get(LocalCache.java:3937) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4739) > at > org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:66) > ... 12 moreCaused by: > org.apache.flink.api.common.InvalidProgramException: Table program cannot be > compiled. This is a bug. Please file an issue.at > org.apache.flink.table.runtime.generated.CompileUtils.doCompile(CompileUtils.java:81) > at > org.apache.flink.table.runtime.generated.CompileUtils.lambda$compile$1(CompileUtils.java:66) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalManualCache$1.load(LocalCache.java:4742) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3527) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2319) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2282) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2197) > ... 15 moreCaused by: org.codehaus.janino.InternalCompilerException: > Compiling "GroupAggsHandler$9687": Code of method > "retract(Lorg/apache/flink/table/dataformat/BaseRow;)V" of
[jira] [Commented] (FLINK-16589) Flink Table SQL fails/crashes with big queries with lots of fields
[ https://issues.apache.org/jira/browse/FLINK-16589?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17059534#comment-17059534 ] Viet Pham commented on FLINK-16589: --- [~libenchao]: please see below an excerpt. The actual query is actually twice as long (which generates the exception above). Yes there is an aggregation, but very simple one. {code:java} SELECT id AS id, LAST_VALUE(platform_domain) AS platform_domain, MAX(now) AS now, SUM(IF((time_slot BETWEEN TIMESTAMPADD(DAY, -MOD(CAST(DAYOFWEEK(now) AS INT) + 5, 7) + 0, now) AND TIMESTAMPADD(SECOND, -1, TIMESTAMPADD(DAY, -MOD(CAST(DAYOFWEEK(now) AS INT) + 5, 7) + 1, now))) AND pws_status = 'delivered', revenue, 0)) AS nmv_trend_WTD0, SUM(IF((time_slot BETWEEN TIMESTAMPADD(DAY, -MOD(CAST(DAYOFWEEK(now) AS INT) + 5, 7) + 1, now) AND TIMESTAMPADD(SECOND, -1, TIMESTAMPADD(DAY, -MOD(CAST(DAYOFWEEK(now) AS INT) + 5, 7) + 2, now))) AND pws_status = 'delivered', revenue, 0)) AS nmv_trend_WTD1, SUM(IF((time_slot BETWEEN TIMESTAMPADD(DAY, -MOD(CAST(DAYOFWEEK(now) AS INT) + 5, 7) + 2, now) AND TIMESTAMPADD(SECOND, -1, TIMESTAMPADD(DAY, -MOD(CAST(DAYOFWEEK(now) AS INT) + 5, 7) + 3, now))) AND pws_status = 'delivered', revenue, 0)) AS nmv_trend_WTD2, SUM(IF((time_slot BETWEEN TIMESTAMPADD(DAY, -MOD(CAST(DAYOFWEEK(now) AS INT) + 5, 7) + 3, now) AND TIMESTAMPADD(SECOND, -1, TIMESTAMPADD(DAY, -MOD(CAST(DAYOFWEEK(now) AS INT) + 5, 7) + 4, now))) AND pws_status = 'delivered', revenue, 0)) AS nmv_trend_WTD3, SUM(IF((time_slot BETWEEN TIMESTAMPADD(DAY, -MOD(CAST(DAYOFWEEK(now) AS INT) + 5, 7) + 4, now) AND TIMESTAMPADD(SECOND, -1, TIMESTAMPADD(DAY, -MOD(CAST(DAYOFWEEK(now) AS INT) + 5, 7) + 5, now))) AND pws_status = 'delivered', revenue, 0)) AS nmv_trend_WTD4, SUM(IF((time_slot BETWEEN TIMESTAMPADD(DAY, -MOD(CAST(DAYOFWEEK(now) AS INT) + 5, 7) + 5, now) AND TIMESTAMPADD(SECOND, -1, TIMESTAMPADD(DAY, -MOD(CAST(DAYOFWEEK(now) AS INT) + 5, 7) + 6, now))) AND pws_status = 'delivered', revenue, 0)) AS nmv_trend_WTD5, SUM(IF((time_slot BETWEEN TIMESTAMPADD(DAY, -MOD(CAST(DAYOFWEEK(now) AS INT) + 5, 7) + 6, now) AND TIMESTAMPADD(SECOND, -1, TIMESTAMPADD(DAY, -MOD(CAST(DAYOFWEEK(now) AS INT) + 5, 7) + 7, now))) AND pws_status = 'delivered', revenue, 0)) AS nmv_trend_WTD6, SUM(IF((time_slot BETWEEN TIMESTAMPADD(HOUR, 0, now) AND TIMESTAMPADD(SECOND, -1, TIMESTAMPADD(HOUR, 1, now))) AND pws_status = 'delivered', revenue, 0)) AS nmv_trend_today_H0, SUM(IF((time_slot BETWEEN TIMESTAMPADD(HOUR, 1, now) AND TIMESTAMPADD(SECOND, -1, TIMESTAMPADD(HOUR, 2, now))) AND pws_status = 'delivered', revenue, 0)) AS nmv_trend_today_H1, SUM(IF((time_slot BETWEEN TIMESTAMPADD(HOUR, 2, now) AND TIMESTAMPADD(SECOND, -1, TIMESTAMPADD(HOUR, 3, now))) AND pws_status = 'delivered', revenue, 0)) AS nmv_trend_today_H2, SUM(IF((time_slot BETWEEN TIMESTAMPADD(HOUR, 3, now) AND TIMESTAMPADD(SECOND, -1, TIMESTAMPADD(HOUR, 4, now))) AND pws_status = 'delivered', revenue, 0)) AS nmv_trend_today_H3, SUM(IF((time_slot BETWEEN TIMESTAMPADD(HOUR, 4, now) AND TIMESTAMPADD(SECOND, -1, TIMESTAMPADD(HOUR, 5, now))) AND pws_status = 'delivered', revenue, 0)) AS nmv_trend_today_H4, SUM(IF((time_slot BETWEEN TIMESTAMPADD(HOUR, 5, now) AND TIMESTAMPADD(SECOND, -1, TIMESTAMPADD(HOUR, 6, now))) AND pws_status = 'delivered', revenue, 0)) AS nmv_trend_today_H5, SUM(IF((time_slot BETWEEN TIMESTAMPADD(HOUR, 6, now) AND TIMESTAMPADD(SECOND, -1, TIMESTAMPADD(HOUR, 7, now))) AND pws_status = 'delivered', revenue, 0)) AS nmv_trend_today_H6, SUM(IF((time_slot BETWEEN TIMESTAMPADD(HOUR, 7, now) AND TIMESTAMPADD(SECOND, -1, TIMESTAMPADD(HOUR, 8, now))) AND pws_status = 'delivered', revenue, 0)) AS nmv_trend_today_H7, SUM(IF((time_slot BETWEEN TIMESTAMPADD(HOUR, 8, now) AND TIMESTAMPADD(SECOND, -1, TIMESTAMPADD(HOUR, 9, now))) AND pws_status = 'delivered', revenue, 0)) AS nmv_trend_today_H8, SUM(IF((time_slot BETWEEN TIMESTAMPADD(HOUR, 9, now) AND TIMESTAMPADD(SECOND, -1, TIMESTAMPADD(HOUR, 10, now))) AND pws_status = 'delivered', revenue, 0)) AS nmv_trend_today_H9, SUM(IF((time_slot BETWEEN TIMESTAMPADD(HOUR, 10, now) AND TIMESTAMPADD(SECOND, -1, TIMESTAMPADD(HOUR, 11, now))) AND pws_status = 'delivered', revenue, 0)) AS nmv_trend_today_H10, SUM(IF((time_slot BETWEEN TIMESTAMPADD(HOUR, 11, now) AND TIMESTAMPADD(SECOND, -1, TIMESTAMPADD(HOUR, 12, now))) AND pws_status = 'delivered', revenue, 0)) AS nmv_trend_today_H11, SUM(IF((time_slot BETWEEN TIMESTAMPADD(HOUR, 12, now) AND TIMESTAMPADD(SECOND, -1, TIMESTAMPADD(HOUR, 13, now))) AND pws_status = 'delivered', revenue, 0)) AS nmv_trend_today_H12, SUM(IF((time_slot BETWEEN TIMESTAMPADD(HOUR, 13, now) AND TIMESTAMPADD(SECOND, -1, TIMESTAMPADD(HOUR, 14, now))) AND pws_status = 'delivered',
[jira] [Commented] (FLINK-16589) Flink Table SQL fails/crashes with big queries with lots of fields
[ https://issues.apache.org/jira/browse/FLINK-16589?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17059533#comment-17059533 ] Benchao Li commented on FLINK-16589: [~jark] Thanks for the assignment. [~StarGhost] Could you show us the query which produces this exception? As you described above, you just did a SELECT, and then the exception is threw? Since the exception says the generated aggregation code is too long, there should be some aggregation in your query. > Flink Table SQL fails/crashes with big queries with lots of fields > -- > > Key: FLINK-16589 > URL: https://issues.apache.org/jira/browse/FLINK-16589 > Project: Flink > Issue Type: Bug > Components: Table SQL / API >Affects Versions: 1.10.0 >Reporter: Viet Pham >Assignee: Benchao Li >Priority: Critical > Fix For: 1.10.1, 1.11.0 > > > Hi, > My use case is a streaming application with a few streaming tables. > I was trying to build a SELECT query (and registering it as a temporary view) > with about 200 fields/expressions out of another streaming table. The > application is successfully submitted to Flink cluster. However the worker > processes keep crashing, with the exception as quoted below. > It clearly mentioned in the log that this is a bug, so I fire this ticket. By > the way, if I lower the number of fields down to 100 then it works nicely. > Please advice. > Thanks a lot for all the efforts bring Flink up. It is really amazing! > {code:java} > java.lang.RuntimeException: Could not instantiate generated class > 'GroupAggsHandler$9687'at > org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:57) > at > org.apache.flink.table.runtime.operators.aggregate.MiniBatchGroupAggFunction.open(MiniBatchGroupAggFunction.java:136) > at > org.apache.flink.table.runtime.operators.bundle.AbstractMapBundleOperator.open(AbstractMapBundleOperator.java:84) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.initializeStateAndOpen(StreamTask.java:1007) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$beforeInvoke$0(StreamTask.java:454) > at > org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.runThrowing(StreamTaskActionExecutor.java:94) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.beforeInvoke(StreamTask.java:449) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:461) > at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:707)at > org.apache.flink.runtime.taskmanager.Task.run(Task.java:532)at > java.lang.Thread.run(Thread.java:748)Caused by: > org.apache.flink.util.FlinkRuntimeException: > org.apache.flink.api.common.InvalidProgramException: Table program cannot be > compiled. This is a bug. Please file an issue.at > org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:68) > at > org.apache.flink.table.runtime.generated.GeneratedClass.compile(GeneratedClass.java:78) > at > org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:52) > ... 10 moreCaused by: > org.apache.flink.shaded.guava18.com.google.common.util.concurrent.UncheckedExecutionException: > org.apache.flink.api.common.InvalidProgramException: Table program cannot be > compiled. This is a bug. Please file an issue.at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2203) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache.get(LocalCache.java:3937) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4739) > at > org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:66) > ... 12 moreCaused by: > org.apache.flink.api.common.InvalidProgramException: Table program cannot be > compiled. This is a bug. Please file an issue.at > org.apache.flink.table.runtime.generated.CompileUtils.doCompile(CompileUtils.java:81) > at > org.apache.flink.table.runtime.generated.CompileUtils.lambda$compile$1(CompileUtils.java:66) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalManualCache$1.load(LocalCache.java:4742) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3527) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2319) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2282) > at >
[jira] [Commented] (FLINK-16589) Flink Table SQL fails/crashes with big queries with lots of fields
[ https://issues.apache.org/jira/browse/FLINK-16589?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17058859#comment-17058859 ] Jark Wu commented on FLINK-16589: - Yes. We didn't split code for GroupAggsHandler, but that can be done as [~libenchao] said, even though it is a complete solution... Assigned this issue to you [~libenchao]. > Flink Table SQL fails/crashes with big queries with lots of fields > -- > > Key: FLINK-16589 > URL: https://issues.apache.org/jira/browse/FLINK-16589 > Project: Flink > Issue Type: Bug > Components: Table SQL / API >Affects Versions: 1.10.0 >Reporter: Viet Pham >Assignee: Benchao Li >Priority: Major > Fix For: 1.10.1, 1.11.0 > > > Hi, > My use case is a streaming application with a few streaming tables. > I was trying to build a SELECT query (and registering it as a temporary view) > with about 200 fields/expressions out of another streaming table. The > application is successfully submitted to Flink cluster. However the worker > processes keep crashing, with the exception as quoted below. > It clearly mentioned in the log that this is a bug, so I fire this ticket. By > the way, if I lower the number of fields down to 100 then it works nicely. > Please advice. > Thanks a lot for all the efforts bring Flink up. It is really amazing! > {code:java} > java.lang.RuntimeException: Could not instantiate generated class > 'GroupAggsHandler$9687'at > org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:57) > at > org.apache.flink.table.runtime.operators.aggregate.MiniBatchGroupAggFunction.open(MiniBatchGroupAggFunction.java:136) > at > org.apache.flink.table.runtime.operators.bundle.AbstractMapBundleOperator.open(AbstractMapBundleOperator.java:84) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.initializeStateAndOpen(StreamTask.java:1007) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$beforeInvoke$0(StreamTask.java:454) > at > org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.runThrowing(StreamTaskActionExecutor.java:94) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.beforeInvoke(StreamTask.java:449) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:461) > at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:707)at > org.apache.flink.runtime.taskmanager.Task.run(Task.java:532)at > java.lang.Thread.run(Thread.java:748)Caused by: > org.apache.flink.util.FlinkRuntimeException: > org.apache.flink.api.common.InvalidProgramException: Table program cannot be > compiled. This is a bug. Please file an issue.at > org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:68) > at > org.apache.flink.table.runtime.generated.GeneratedClass.compile(GeneratedClass.java:78) > at > org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:52) > ... 10 moreCaused by: > org.apache.flink.shaded.guava18.com.google.common.util.concurrent.UncheckedExecutionException: > org.apache.flink.api.common.InvalidProgramException: Table program cannot be > compiled. This is a bug. Please file an issue.at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2203) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache.get(LocalCache.java:3937) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4739) > at > org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:66) > ... 12 moreCaused by: > org.apache.flink.api.common.InvalidProgramException: Table program cannot be > compiled. This is a bug. Please file an issue.at > org.apache.flink.table.runtime.generated.CompileUtils.doCompile(CompileUtils.java:81) > at > org.apache.flink.table.runtime.generated.CompileUtils.lambda$compile$1(CompileUtils.java:66) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalManualCache$1.load(LocalCache.java:4742) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3527) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2319) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2282) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2197) > ... 15 moreCaused by: org.codehaus.janino.InternalCompilerException: > Compiling
[jira] [Commented] (FLINK-16589) Flink Table SQL fails/crashes with big queries with lots of fields
[ https://issues.apache.org/jira/browse/FLINK-16589?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17058745#comment-17058745 ] Benchao Li commented on FLINK-16589: [~ykt836] We did the code split only for CalcCodeGenerator in https://issues.apache.org/jira/browse/FLINK-15430. If we need to split for {{GroupAggsHandler}} too, I can help to do that. > Flink Table SQL fails/crashes with big queries with lots of fields > -- > > Key: FLINK-16589 > URL: https://issues.apache.org/jira/browse/FLINK-16589 > Project: Flink > Issue Type: Bug > Components: Table SQL / API >Affects Versions: 1.10.0 >Reporter: Viet Pham >Priority: Major > > Hi, > My use case is a streaming application with a few streaming tables. > I was trying to build a SELECT query (and registering it as a temporary view) > with about 200 fields/expressions out of another streaming table. The > application is successfully submitted to Flink cluster. However the worker > processes keep crashing, with the exception as quoted below. > It clearly mentioned in the log that this is a bug, so I fire this ticket. By > the way, if I lower the number of fields down to 100 then it works nicely. > Please advice. > Thanks a lot for all the efforts bring Flink up. It is really amazing! > {code:java} > java.lang.RuntimeException: Could not instantiate generated class > 'GroupAggsHandler$9687'at > org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:57) > at > org.apache.flink.table.runtime.operators.aggregate.MiniBatchGroupAggFunction.open(MiniBatchGroupAggFunction.java:136) > at > org.apache.flink.table.runtime.operators.bundle.AbstractMapBundleOperator.open(AbstractMapBundleOperator.java:84) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.initializeStateAndOpen(StreamTask.java:1007) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$beforeInvoke$0(StreamTask.java:454) > at > org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.runThrowing(StreamTaskActionExecutor.java:94) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.beforeInvoke(StreamTask.java:449) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:461) > at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:707)at > org.apache.flink.runtime.taskmanager.Task.run(Task.java:532)at > java.lang.Thread.run(Thread.java:748)Caused by: > org.apache.flink.util.FlinkRuntimeException: > org.apache.flink.api.common.InvalidProgramException: Table program cannot be > compiled. This is a bug. Please file an issue.at > org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:68) > at > org.apache.flink.table.runtime.generated.GeneratedClass.compile(GeneratedClass.java:78) > at > org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:52) > ... 10 moreCaused by: > org.apache.flink.shaded.guava18.com.google.common.util.concurrent.UncheckedExecutionException: > org.apache.flink.api.common.InvalidProgramException: Table program cannot be > compiled. This is a bug. Please file an issue.at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2203) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache.get(LocalCache.java:3937) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4739) > at > org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:66) > ... 12 moreCaused by: > org.apache.flink.api.common.InvalidProgramException: Table program cannot be > compiled. This is a bug. Please file an issue.at > org.apache.flink.table.runtime.generated.CompileUtils.doCompile(CompileUtils.java:81) > at > org.apache.flink.table.runtime.generated.CompileUtils.lambda$compile$1(CompileUtils.java:66) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalManualCache$1.load(LocalCache.java:4742) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3527) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2319) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2282) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2197) > ... 15 moreCaused by: org.codehaus.janino.InternalCompilerException: > Compiling "GroupAggsHandler$9687": Code of method >
[jira] [Commented] (FLINK-16589) Flink Table SQL fails/crashes with big queries with lots of fields
[ https://issues.apache.org/jira/browse/FLINK-16589?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17058742#comment-17058742 ] Jingsong Lee commented on FLINK-16589: -- If just too many fields in "select *" with "group by", it can be resolved, just like `ProjectionCodeGenerator`, we can extract its "loop" code generation. > Flink Table SQL fails/crashes with big queries with lots of fields > -- > > Key: FLINK-16589 > URL: https://issues.apache.org/jira/browse/FLINK-16589 > Project: Flink > Issue Type: Bug > Components: Table SQL / API >Affects Versions: 1.10.0 >Reporter: Viet Pham >Priority: Major > > Hi, > My use case is a streaming application with a few streaming tables. > I was trying to build a SELECT query (and registering it as a temporary view) > with about 200 fields/expressions out of another streaming table. The > application is successfully submitted to Flink cluster. However the worker > processes keep crashing, with the exception as quoted below. > It clearly mentioned in the log that this is a bug, so I fire this ticket. By > the way, if I lower the number of fields down to 100 then it works nicely. > Please advice. > Thanks a lot for all the efforts bring Flink up. It is really amazing! > {code:java} > java.lang.RuntimeException: Could not instantiate generated class > 'GroupAggsHandler$9687'at > org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:57) > at > org.apache.flink.table.runtime.operators.aggregate.MiniBatchGroupAggFunction.open(MiniBatchGroupAggFunction.java:136) > at > org.apache.flink.table.runtime.operators.bundle.AbstractMapBundleOperator.open(AbstractMapBundleOperator.java:84) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.initializeStateAndOpen(StreamTask.java:1007) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$beforeInvoke$0(StreamTask.java:454) > at > org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.runThrowing(StreamTaskActionExecutor.java:94) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.beforeInvoke(StreamTask.java:449) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:461) > at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:707)at > org.apache.flink.runtime.taskmanager.Task.run(Task.java:532)at > java.lang.Thread.run(Thread.java:748)Caused by: > org.apache.flink.util.FlinkRuntimeException: > org.apache.flink.api.common.InvalidProgramException: Table program cannot be > compiled. This is a bug. Please file an issue.at > org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:68) > at > org.apache.flink.table.runtime.generated.GeneratedClass.compile(GeneratedClass.java:78) > at > org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:52) > ... 10 moreCaused by: > org.apache.flink.shaded.guava18.com.google.common.util.concurrent.UncheckedExecutionException: > org.apache.flink.api.common.InvalidProgramException: Table program cannot be > compiled. This is a bug. Please file an issue.at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2203) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache.get(LocalCache.java:3937) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4739) > at > org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:66) > ... 12 moreCaused by: > org.apache.flink.api.common.InvalidProgramException: Table program cannot be > compiled. This is a bug. Please file an issue.at > org.apache.flink.table.runtime.generated.CompileUtils.doCompile(CompileUtils.java:81) > at > org.apache.flink.table.runtime.generated.CompileUtils.lambda$compile$1(CompileUtils.java:66) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalManualCache$1.load(LocalCache.java:4742) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3527) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2319) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2282) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2197) > ... 15 moreCaused by: org.codehaus.janino.InternalCompilerException: > Compiling "GroupAggsHandler$9687": Code of method >
[jira] [Commented] (FLINK-16589) Flink Table SQL fails/crashes with big queries with lots of fields
[ https://issues.apache.org/jira/browse/FLINK-16589?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17058673#comment-17058673 ] Kurt Young commented on FLINK-16589: The root cause is here: Code of method "retract(Lorg/apache/flink/table/dataformat/BaseRow;)V" of class "GroupAggsHandler$9687" grows beyond 64 KB We didn't do proper code split when generated code is too long. cc [~jark] > Flink Table SQL fails/crashes with big queries with lots of fields > -- > > Key: FLINK-16589 > URL: https://issues.apache.org/jira/browse/FLINK-16589 > Project: Flink > Issue Type: Bug > Components: Table SQL / API >Affects Versions: 1.10.0 >Reporter: Viet Pham >Priority: Major > > Hi, > My use case is a streaming application with a few streaming tables. > I was trying to build a SELECT query (and registering it as a temporary view) > with about 200 fields/expressions out of another streaming table. The > application is successfully submitted to Flink cluster. However the worker > processes keep crashing, with the exception as quoted below. > It clearly mentioned in the log that this is a bug, so I fire this ticket. By > the way, if I lower the number of fields down to 100 then it works nicely. > Please advice. > Thanks a lot for all the efforts bring Flink up. It is really amazing! > {code:java} > java.lang.RuntimeException: Could not instantiate generated class > 'GroupAggsHandler$9687'at > org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:57) > at > org.apache.flink.table.runtime.operators.aggregate.MiniBatchGroupAggFunction.open(MiniBatchGroupAggFunction.java:136) > at > org.apache.flink.table.runtime.operators.bundle.AbstractMapBundleOperator.open(AbstractMapBundleOperator.java:84) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.initializeStateAndOpen(StreamTask.java:1007) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$beforeInvoke$0(StreamTask.java:454) > at > org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.runThrowing(StreamTaskActionExecutor.java:94) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.beforeInvoke(StreamTask.java:449) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:461) > at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:707)at > org.apache.flink.runtime.taskmanager.Task.run(Task.java:532)at > java.lang.Thread.run(Thread.java:748)Caused by: > org.apache.flink.util.FlinkRuntimeException: > org.apache.flink.api.common.InvalidProgramException: Table program cannot be > compiled. This is a bug. Please file an issue.at > org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:68) > at > org.apache.flink.table.runtime.generated.GeneratedClass.compile(GeneratedClass.java:78) > at > org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:52) > ... 10 moreCaused by: > org.apache.flink.shaded.guava18.com.google.common.util.concurrent.UncheckedExecutionException: > org.apache.flink.api.common.InvalidProgramException: Table program cannot be > compiled. This is a bug. Please file an issue.at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2203) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache.get(LocalCache.java:3937) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4739) > at > org.apache.flink.table.runtime.generated.CompileUtils.compile(CompileUtils.java:66) > ... 12 moreCaused by: > org.apache.flink.api.common.InvalidProgramException: Table program cannot be > compiled. This is a bug. Please file an issue.at > org.apache.flink.table.runtime.generated.CompileUtils.doCompile(CompileUtils.java:81) > at > org.apache.flink.table.runtime.generated.CompileUtils.lambda$compile$1(CompileUtils.java:66) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LocalManualCache$1.load(LocalCache.java:4742) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3527) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2319) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2282) > at > org.apache.flink.shaded.guava18.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2197) > ... 15 moreCaused by: org.codehaus.janino.InternalCompilerException: > Compiling