Hi, u...@flink.apache.org is used for question discussions, dev@flink.apache.org is used for development related discussions.
Regarding the exception, that indicates flink-table-blink_xx.jar is not in the classpath, please check it. Best, Godfrey 魏烽 <weif...@nequal.com> 于2020年8月5日周三 下午5:15写道: > Hi > > I have a question when use flink sql 1.11.0 > > Error: > > Exception in thread "main" > org.apache.flink.table.api.NoMatchingTableFactoryException: Could not find > a suitable table factory for > 'org.apache.flink.table.delegation.ExecutorFactory' in > > the classpath. > > > Reason: No factory implements > 'org.apache.flink.table.delegation.ExecutorFactory'. > > > The following properties are requested: > > class-name=org.apache.flink.table.planner.delegation.BlinkExecutorFactory > > streaming-mode=true > > > The following factories have been considered: > > org.apache.flink.table.module.CoreModuleFactory > > org.apache.flink.table.sources.CsvBatchTableSourceFactory > > org.apache.flink.table.sources.CsvAppendTableSourceFactory > > org.apache.flink.table.sinks.CsvBatchTableSinkFactory > > org.apache.flink.table.sinks.CsvAppendTableSinkFactory > > org.apache.flink.table.catalog.GenericInMemoryCatalogFactory > > org.apache.flink.table.filesystem.FileSystemTableFactory > > org.apache.flink.api.java.io.jdbc.JDBCTableSourceSinkFactory > > org.apache.flink.connector.jdbc.catalog.factory.JdbcCatalogFactory > > org.apache.flink.connector.jdbc.table.JdbcTableSourceSinkFactory > > at > org.apache.flink.table.factories.TableFactoryService.filterByFactoryClass(TableFactoryService.java:238) > > at > org.apache.flink.table.factories.TableFactoryService.filter(TableFactoryService.java:185) > > at > org.apache.flink.table.factories.TableFactoryService.findAllInternal(TableFactoryService.java:171) > > at > org.apache.flink.table.factories.TableFactoryService.findAll(TableFactoryService.java:125) > > at > org.apache.flink.table.factories.ComponentFactoryService.find(ComponentFactoryService.java:48) > > at > org.apache.flink.table.api.internal.TableEnvironmentImpl.create(TableEnvironmentImpl.java:257) > > at > org.apache.flink.table.api.TableEnvironment.create(TableEnvironment.java:93) > > at com.weifeng.demo.JoinDemo04$.main(JoinDemo04.scala:15) > > at com.weifeng.demo.JoinDemo04.main(JoinDemo04.scala) > > > > My code is: > > val settings = > EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build() > > val tableEnv = TableEnvironment.create(settings) > val mapedStream: DataStream[util.Map[String, Any]] = > environment.readTextFile("/Users/weifeng/IdeaProjects/flinksqldemo/src/main/resources/demo.txt", > "utf-8") > .map(x => { > val splits = x.split(",", -1) > var map1: util.Map[String, Any] = new util.LinkedHashMap[String, Any]() > map1.put("mobile_id", splits(0)) > map1.put("name", splits(1)) > map1 > }) > > val mapedStream1: DataStream[util.Map[String, Any]] = > environment.readTextFile("/Users/weifeng/IdeaProjects/flinksqldemo/src/main/resources/demo.txt", > "utf-8") > .map(x => { > val splits = x.split(",", -1) > var map1: util.Map[String, Any] = new util.LinkedHashMap[String, Any]() > map1.put("mobile_id", splits(0)) > map1.put("name", splits(1)) > map1 > }) > > > val name = "mypg" > val defaultDatabase = "bdhtest" > val username = "bdh" > val password = "flRfRO9Lovgc" > val baseUrl = "jdbc:postgresql://10.218.1.190:8433/bdhtest” > val catalog = new JdbcCatalog(name, defaultDatabase, username, password, > baseUrl) > tableEnv.registerCatalog("mypg", catalog) > tableEnv.useCatalog("mypg") > tableEnv.execute("JoinDemo04”) > > Thank You > 原始邮件 > 发件人: Caizhi Weng (Jira)<j...@apache.org> > 收件人: dev<dev@flink.apache.org> > 发送时间: 2020年8月5日(周三) 14:59 > 主题: [jira] [Created] (FLINK-18821) Netty client retry mechanism maycause > PartitionRequestClientFactory#createPartitionRequestClient to waitinfinitely > > > Caizhi Weng created FLINK-18821: > ----------------------------------- > > Summary: Netty client retry mechanism may cause > PartitionRequestClientFactory#createPartitionRequestClient to wait > infinitely > Key: FLINK-18821 > URL: https://issues.apache.org/jira/browse/FLINK-18821 > Project: Flink > Issue Type: Bug > Components: Runtime / Network > Affects Versions: 1.11.0 > Reporter: Caizhi Weng > > > When running TPCDS 10T benchmark on Flink I found some of the task slots > stuck. After some investigation there seems to be a bug in > {{PartitionRequestClientFactory}}. > > When a task tries to require a partition of data from its upstream task > but fails, {{PartitionRequestClientFactory#connect}} will throw > {{RemoteTransportException}} and > {{PartitionRequestClientFactory#connectWithRetries}} will throw > {{CompletionException}}. However this exception is not caught by > {{PartitionRequestClientFactory#connect}} and it will eventually fail the > task. > > But {{PartitionRequestClientFactory}} lives in a task manager not in a > task. In {{PartitionRequestClientFactory}} a {{ConcurrentHashMap}} named > {{clients}} is maintained for reusing {{NettyPartitionRequestClient}}. When > the above exception happens, {{clients}} is not cleaned up; When the next > call to {{PartitionRequestClientFactory#connect}} with the same connection > id comes, it will use the invalid {{CompletableFuture}} in {{clients}} and > this future will never complete, causing the task to stuck forever. > > Exception stack: > {code} > 2020-08-05 03:37:07,539 ERROR > org.apache.flink.runtime.io.network.netty.PartitionRequestClientFactory > [] - Failed 1 times to connect to <host-name>/<ip>:<port> > org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException: > Connecting to remote task manager '<host-name>/<ip>:<port>' has failed. > This might indicate that the remote task manager has been lost. > at > org.apache.flink.runtime.io.network.netty.PartitionRequestClientFactory.connect(PartitionRequestClientFactory.java:120) > ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] > at > org.apache.flink.runtime.io.network.netty.PartitionRequestClientFactory.connectWithRetries(PartitionRequestClientFactory.java:99) > ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] > at > org.apache.flink.runtime.io.network.netty.PartitionRequestClientFactory.createPartitionRequestClient(PartitionRequestClientFactory.java:76) > ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] > at > org.apache.flink.runtime.io.network.netty.NettyConnectionManager.createPartitionRequestClient(NettyConnectionManager.java:67) > ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] > at > org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel.requestSubpartition(RemoteInputChannel.java:146) > ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] > at > org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.internalRequestPartitions(SingleInputGate.java:329) > ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] > at > org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.requestPartitions(SingleInputGate.java:301) > ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] > at > org.apache.flink.runtime.taskmanager.InputGateWithMetrics.requestPartitions(InputGateWithMetrics.java:95) > ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] > at > org.apache.flink.streaming.runtime.tasks.StreamTask.requestPartitions(StreamTask.java:514) > ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] > at > org.apache.flink.streaming.runtime.tasks.StreamTask.readRecoveredChannelState(StreamTask.java:484) > ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] > at > org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$beforeInvoke$0(StreamTask.java:475) > ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] > at > org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.runThrowing(StreamTaskActionExecutor.java:47) > [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] > at > org.apache.flink.streaming.runtime.tasks.StreamTask.beforeInvoke(StreamTask.java:469) > [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] > at > org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:522) > [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] > at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:721) > [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] > at org.apache.flink.runtime.taskmanager.Task.run(Task.java:546) > [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] > at java.lang.Thread.run(Thread.java:834) [?:1.8.0_102] > Caused by: java.lang.NullPointerException > at > org.apache.flink.util.Preconditions.checkNotNull(Preconditions.java:58) > ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] > at > org.apache.flink.runtime.io.network.netty.NettyPartitionRequestClient.<init>(NettyPartitionRequestClient.java:73) > ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] > at > org.apache.flink.runtime.io.network.netty.PartitionRequestClientFactory.connect(PartitionRequestClientFactory.java:114) > ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] > ... 16 more > {code} > > > > > -- > This message was sent by Atlassian Jira > (v8.3.4#803005) > >