[ https://issues.apache.org/jira/browse/FLINK-30001?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17633497#comment-17633497 ]
xiaohang.li commented on FLINK-30001: ------------------------------------- 经查询, 默认情况下 flink 中的 org.apache.flink.table.planner.loader.PlannerModule 模块使用 /tmp 目录来作为临时的工作路径,因此会尝试调用 jave 的 java.nio.file.Files 类来创建这个目录,但是如果 /tmp 目录是一个指向 /mnt/tmp 的符号软链接,这种情况 java.nio.file.Files 类无法处理,从而导致出现报错。需要在sql-client.sh添加临时路径的配置: export JVM_ARGS="-Djava.io.tmpdir=/mnt/tmp > sql-client.sh start failed > -------------------------- > > Key: FLINK-30001 > URL: https://issues.apache.org/jira/browse/FLINK-30001 > Project: Flink > Issue Type: Bug > Components: Command Line Client > Affects Versions: 1.16.0, 1.15.2 > Reporter: xiaohang.li > Priority: Major > > [hadoop@master flink-1.15.0]$ ./bin/sql-client.sh > Setting HADOOP_CONF_DIR=/etc/hadoop/conf because no HADOOP_CONF_DIR or > HADOOP_CLASSPATH was set. > Setting HBASE_CONF_DIR=/etc/hbase/conf because no HBASE_CONF_DIR was set. > Exception in thread "main" org.apache.flink.table.client.SqlClientException: > Unexpected exception. This is a bug. Please consider filing an issue. > at > org.apache.flink.table.client.SqlClient.startClient(SqlClient.java:201) > at org.apache.flink.table.client.SqlClient.main(SqlClient.java:161) > Caused by: org.apache.flink.table.api.TableException: Could not instantiate > the executor. Make sure a planner module is on the classpath > at > org.apache.flink.table.client.gateway.context.ExecutionContext.lookupExecutor(ExecutionContext.java:163) > at > org.apache.flink.table.client.gateway.context.ExecutionContext.createTableEnvironment(ExecutionContext.java:111) > at > org.apache.flink.table.client.gateway.context.ExecutionContext.<init>(ExecutionContext.java:66) > at > org.apache.flink.table.client.gateway.context.SessionContext.create(SessionContext.java:247) > at > org.apache.flink.table.client.gateway.local.LocalContextUtils.buildSessionContext(LocalContextUtils.java:87) > at > org.apache.flink.table.client.gateway.local.LocalExecutor.openSession(LocalExecutor.java:87) > at org.apache.flink.table.client.SqlClient.start(SqlClient.java:88) > at > org.apache.flink.table.client.SqlClient.startClient(SqlClient.java:187) > ... 1 more > Caused by: org.apache.flink.table.api.TableException: Unexpected error when > trying to load service provider for factories. > at > org.apache.flink.table.factories.FactoryUtil.lambda$discoverFactories$19(FactoryUtil.java:813) > at java.util.ArrayList.forEach(ArrayList.java:1259) > at > org.apache.flink.table.factories.FactoryUtil.discoverFactories(FactoryUtil.java:799) > at > org.apache.flink.table.factories.FactoryUtil.discoverFactory(FactoryUtil.java:517) > at > org.apache.flink.table.client.gateway.context.ExecutionContext.lookupExecutor(ExecutionContext.java:154) > ... 8 more > Caused by: java.util.ServiceConfigurationError: > org.apache.flink.table.factories.Factory: Provider > org.apache.flink.table.planner.loader.DelegateExecutorFactory could not be > instantiated > at java.util.ServiceLoader.fail(ServiceLoader.java:232) > at java.util.ServiceLoader.access$100(ServiceLoader.java:185) > at > java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:384) > at java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:404) > at java.util.ServiceLoader$1.next(ServiceLoader.java:480) > at > org.apache.flink.table.factories.ServiceLoaderUtil.load(ServiceLoaderUtil.java:42) > at > org.apache.flink.table.factories.FactoryUtil.discoverFactories(FactoryUtil.java:798) > ... 10 more > Caused by: java.lang.ExceptionInInitializerError > at > org.apache.flink.table.planner.loader.PlannerModule.getInstance(PlannerModule.java:135) > at > org.apache.flink.table.planner.loader.DelegateExecutorFactory.<init>(DelegateExecutorFactory.java:34) > at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native > Method) > at > sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62) > at > sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) > at java.lang.reflect.Constructor.newInstance(Constructor.java:423) > at java.lang.Class.newInstance(Class.java:442) > at > java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:380) > ... 14 more > Caused by: org.apache.flink.table.api.TableException: Could not initialize > the table planner components loader. > at > org.apache.flink.table.planner.loader.PlannerModule.<init>(PlannerModule.java:123) > at > org.apache.flink.table.planner.loader.PlannerModule.<init>(PlannerModule.java:52) > at > org.apache.flink.table.planner.loader.PlannerModule$PlannerComponentsHolder.<clinit>(PlannerModule.java:131) > ... 22 more > Caused by: java.nio.file.FileAlreadyExistsException: /tmp > at > sun.nio.fs.UnixException.translateToIOException(UnixException.java:88) > at > sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:102) > at > sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:107) > at > sun.nio.fs.UnixFileSystemProvider.createDirectory(UnixFileSystemProvider.java:384) > at java.nio.file.Files.createDirectory(Files.java:674) > at java.nio.file.Files.createAndCheckIsDirectory(Files.java:781) > at java.nio.file.Files.createDirectories(Files.java:727) > at > org.apache.flink.table.planner.loader.PlannerModule.<init>(PlannerModule.java:96) > ... 24 more -- This message was sent by Atlassian Jira (v8.20.10#820010)