You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user-zh@flink.apache.org by ruiyun wan <wr...@gmail.com> on 2022/04/26 06:41:19 UTC

关于使用SQL Client提交执行目标为yarn-per-job的作业时无日志文件(jobmanager.log)的求助

Flink版本:1.13
问题描述:使用sql-client.sh启动yarn-per-job(execution.target =
yarn-per-job)时,在YARN集群侧生成的launch_container.sh中,启动org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint的参数中无-Dlog.file和-Dlog4j.configuration属性参数,导致没有jobmanager.log日志文件,如何设置才能影响Yarn生成的launch_container.sh包含上述参数。

Re: 关于使用SQL Client提交执行目标为yarn-per-job的作业时无日志文件(jobmanager.log)的求助

Posted by Qishang <zh...@gmail.com>.
Hi.
FYI : https://issues.apache.org/jira/browse/FLINK-25435

Qishang <zh...@gmail.com> 于2022年4月27日周三 17:28写道:

> Hi .
> 代码好像是没有设置
> 用这个手动设置一下
> set $internal.deployment.config-dir=/opt/flink-1.14.3/conf
>
> 调用链路的话,凑合看一下吧。下面就是获取到 YarnJobClusterExecutorFactory  ->
> YarnJobClusterExecutor -> YarnClusterClientFactory -> YarnClusterDescriptor
>
> getExecutorFactory:58, DefaultExecutorServiceLoader
> (org.apache.flink.core.execution)
> executeAsync:2032, StreamExecutionEnvironment
> (org.apache.flink.streaming.api.environment)
> executeAsync:95, DefaultExecutor
> (org.apache.flink.table.planner.delegation)
> executeQueryOperation:811, TableEnvironmentImpl
> (org.apache.flink.table.api.internal)
> executeInternal:1274, TableEnvironmentImpl
> (org.apache.flink.table.api.internal)
> lambda$executeOperation$3:209, LocalExecutor
> (org.apache.flink.table.client.gateway.local)
> wrapClassLoader:88, ExecutionContext
> (org.apache.flink.table.client.gateway.context)
> executeOperation:209, LocalExecutor
> (org.apache.flink.table.client.gateway.local)
> executeQuery:231, LocalExecutor
> (org.apache.flink.table.client.gateway.local)
> callSelect:532, CliClient (org.apache.flink.table.client.cli)
> callOperation:423, CliClient (org.apache.flink.table.client.cli)
> lambda$executeStatement$1:332, CliClient
> (org.apache.flink.table.client.cli)
> executeStatement:325, CliClient (org.apache.flink.table.client.cli)
> executeInteractive:297, CliClient (org.apache.flink.table.client.cli)
> executeInInteractiveMode:221, CliClient (org.apache.flink.table.client.cli)
> openCli:151, SqlClient (org.apache.flink.table.client)
> start:95, SqlClient (org.apache.flink.table.client)
> startClient:187, SqlClient (org.apache.flink.table.client)
> main:161, SqlClient (org.apache.flink.table.client)
>
> ruiyun wan <wr...@gmail.com> 于2022年4月27日周三 14:51写道:
>
>> 这个必须有,因为用yarn-session.sh创建集群会有jobmanager.log。能够找到yarn-session.sh的启动类(org.apache.flink.yarn.cli.FlinkYarnSessionCli)到
>> YarnClusterDescriptor的调用路径。
>> [image: image.png]
>> 但是我没有找到从sql-client.sh的启动类(org.apache.flink.table.client.SqlClient)到
>> YarnClusterDescriptor的调用路径。这两者不在同一个包。
>>
>> Qishang <zh...@gmail.com> 于2022年4月27日周三 13:46写道:
>>
>>> Hi.
>>> 确认下 conf 下,是否有 log4j.properties
>>>
>>> 应该是在这个地放生成的,
>>>
>>> https://github.com/apache/flink/blob/release-1.13/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java#L1699
>>>
>>>
>>> ruiyun wan <wr...@gmail.com> 于2022年4月26日周二 14:41写道:
>>>
>>> > Flink版本:1.13
>>> > 问题描述:使用sql-client.sh启动yarn-per-job(execution.target =
>>> >
>>> >
>>> yarn-per-job)时,在YARN集群侧生成的launch_container.sh中,启动org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint的参数中无-Dlog.file和-Dlog4j.configuration属性参数,导致没有jobmanager.log日志文件,如何设置才能影响Yarn生成的launch_container.sh包含上述参数。
>>> >
>>>
>>

Re: 关于使用SQL Client提交执行目标为yarn-per-job的作业时无日志文件(jobmanager.log)的求助

Posted by ruiyun wan <wr...@gmail.com>.
我也是刚看到YarnLogConfigUtil有这么一段代码:
    public static String getLoggingYarnCommand(final Configuration
configuration) {
        checkNotNull(configuration);

        final String logConfigFilePath =

configuration.getString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE);
        if (logConfigFilePath == null) {
            return "";
        }

        String logCommand = getLog4jCommand(logConfigFilePath);
        if (logCommand.isEmpty()) {
            logCommand = getLogBackCommand(logConfigFilePath);
        }
        return logCommand;
    }
但是YarnConfigOptionsInternal貌似不让用户设置,加了单引号才可以。应该能解决这个问题了。
谢谢。

Qishang <zh...@gmail.com> 于2022年4月27日周三 17:35写道:

> Hi .
> 代码好像是没有设置
> 用这个手动设置一下
> set $internal.deployment.config-dir=/opt/flink-1.14.3/conf
>
> 调用链路的话,凑合看一下吧。下面就是获取到 YarnJobClusterExecutorFactory  ->
> YarnJobClusterExecutor -> YarnClusterClientFactory -> YarnClusterDescriptor
>
> getExecutorFactory:58, DefaultExecutorServiceLoader
> (org.apache.flink.core.execution)
> executeAsync:2032, StreamExecutionEnvironment
> (org.apache.flink.streaming.api.environment)
> executeAsync:95, DefaultExecutor
> (org.apache.flink.table.planner.delegation)
> executeQueryOperation:811, TableEnvironmentImpl
> (org.apache.flink.table.api.internal)
> executeInternal:1274, TableEnvironmentImpl
> (org.apache.flink.table.api.internal)
> lambda$executeOperation$3:209, LocalExecutor
> (org.apache.flink.table.client.gateway.local)
> wrapClassLoader:88, ExecutionContext
> (org.apache.flink.table.client.gateway.context)
> executeOperation:209, LocalExecutor
> (org.apache.flink.table.client.gateway.local)
> executeQuery:231, LocalExecutor
> (org.apache.flink.table.client.gateway.local)
> callSelect:532, CliClient (org.apache.flink.table.client.cli)
> callOperation:423, CliClient (org.apache.flink.table.client.cli)
> lambda$executeStatement$1:332, CliClient
> (org.apache.flink.table.client.cli)
> executeStatement:325, CliClient (org.apache.flink.table.client.cli)
> executeInteractive:297, CliClient (org.apache.flink.table.client.cli)
> executeInInteractiveMode:221, CliClient (org.apache.flink.table.client.cli)
> openCli:151, SqlClient (org.apache.flink.table.client)
> start:95, SqlClient (org.apache.flink.table.client)
> startClient:187, SqlClient (org.apache.flink.table.client)
> main:161, SqlClient (org.apache.flink.table.client)
>
> ruiyun wan <wr...@gmail.com> 于2022年4月27日周三 14:51写道:
>
> >
> 这个必须有,因为用yarn-session.sh创建集群会有jobmanager.log。能够找到yarn-session.sh的启动类(org.apache.flink.yarn.cli.FlinkYarnSessionCli)到
> > YarnClusterDescriptor的调用路径。
> > [image: image.png]
> > 但是我没有找到从sql-client.sh的启动类(org.apache.flink.table.client.SqlClient)到
> > YarnClusterDescriptor的调用路径。这两者不在同一个包。
> >
> > Qishang <zh...@gmail.com> 于2022年4月27日周三 13:46写道:
> >
> >> Hi.
> >> 确认下 conf 下,是否有 log4j.properties
> >>
> >> 应该是在这个地放生成的,
> >>
> >>
> https://github.com/apache/flink/blob/release-1.13/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java#L1699
> >>
> >>
> >> ruiyun wan <wr...@gmail.com> 于2022年4月26日周二 14:41写道:
> >>
> >> > Flink版本:1.13
> >> > 问题描述:使用sql-client.sh启动yarn-per-job(execution.target =
> >> >
> >> >
> >>
> yarn-per-job)时,在YARN集群侧生成的launch_container.sh中,启动org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint的参数中无-Dlog.file和-Dlog4j.configuration属性参数,导致没有jobmanager.log日志文件,如何设置才能影响Yarn生成的launch_container.sh包含上述参数。
> >> >
> >>
> >
>

Re: 关于使用SQL Client提交执行目标为yarn-per-job的作业时无日志文件(jobmanager.log)的求助

Posted by Qishang <zh...@gmail.com>.
Hi .
代码好像是没有设置
用这个手动设置一下
set $internal.deployment.config-dir=/opt/flink-1.14.3/conf

调用链路的话,凑合看一下吧。下面就是获取到 YarnJobClusterExecutorFactory  ->
YarnJobClusterExecutor -> YarnClusterClientFactory -> YarnClusterDescriptor

getExecutorFactory:58, DefaultExecutorServiceLoader
(org.apache.flink.core.execution)
executeAsync:2032, StreamExecutionEnvironment
(org.apache.flink.streaming.api.environment)
executeAsync:95, DefaultExecutor (org.apache.flink.table.planner.delegation)
executeQueryOperation:811, TableEnvironmentImpl
(org.apache.flink.table.api.internal)
executeInternal:1274, TableEnvironmentImpl
(org.apache.flink.table.api.internal)
lambda$executeOperation$3:209, LocalExecutor
(org.apache.flink.table.client.gateway.local)
wrapClassLoader:88, ExecutionContext
(org.apache.flink.table.client.gateway.context)
executeOperation:209, LocalExecutor
(org.apache.flink.table.client.gateway.local)
executeQuery:231, LocalExecutor
(org.apache.flink.table.client.gateway.local)
callSelect:532, CliClient (org.apache.flink.table.client.cli)
callOperation:423, CliClient (org.apache.flink.table.client.cli)
lambda$executeStatement$1:332, CliClient (org.apache.flink.table.client.cli)
executeStatement:325, CliClient (org.apache.flink.table.client.cli)
executeInteractive:297, CliClient (org.apache.flink.table.client.cli)
executeInInteractiveMode:221, CliClient (org.apache.flink.table.client.cli)
openCli:151, SqlClient (org.apache.flink.table.client)
start:95, SqlClient (org.apache.flink.table.client)
startClient:187, SqlClient (org.apache.flink.table.client)
main:161, SqlClient (org.apache.flink.table.client)

ruiyun wan <wr...@gmail.com> 于2022年4月27日周三 14:51写道:

> 这个必须有,因为用yarn-session.sh创建集群会有jobmanager.log。能够找到yarn-session.sh的启动类(org.apache.flink.yarn.cli.FlinkYarnSessionCli)到
> YarnClusterDescriptor的调用路径。
> [image: image.png]
> 但是我没有找到从sql-client.sh的启动类(org.apache.flink.table.client.SqlClient)到
> YarnClusterDescriptor的调用路径。这两者不在同一个包。
>
> Qishang <zh...@gmail.com> 于2022年4月27日周三 13:46写道:
>
>> Hi.
>> 确认下 conf 下,是否有 log4j.properties
>>
>> 应该是在这个地放生成的,
>>
>> https://github.com/apache/flink/blob/release-1.13/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java#L1699
>>
>>
>> ruiyun wan <wr...@gmail.com> 于2022年4月26日周二 14:41写道:
>>
>> > Flink版本:1.13
>> > 问题描述:使用sql-client.sh启动yarn-per-job(execution.target =
>> >
>> >
>> yarn-per-job)时,在YARN集群侧生成的launch_container.sh中,启动org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint的参数中无-Dlog.file和-Dlog4j.configuration属性参数,导致没有jobmanager.log日志文件,如何设置才能影响Yarn生成的launch_container.sh包含上述参数。
>> >
>>
>

Re: 关于使用SQL Client提交执行目标为yarn-per-job的作业时无日志文件(jobmanager.log)的求助

Posted by ruiyun wan <wr...@gmail.com>.
这个必须有,因为用yarn-session.sh创建集群会有jobmanager.log。能够找到yarn-session.sh的启动类(org.apache.flink.yarn.cli.FlinkYarnSessionCli)到
YarnClusterDescriptor的调用路径。
[image: image.png]
但是我没有找到从sql-client.sh的启动类(org.apache.flink.table.client.SqlClient)到
YarnClusterDescriptor的调用路径。这两者不在同一个包。

Qishang <zh...@gmail.com> 于2022年4月27日周三 13:46写道:

> Hi.
> 确认下 conf 下,是否有 log4j.properties
>
> 应该是在这个地放生成的,
>
> https://github.com/apache/flink/blob/release-1.13/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java#L1699
>
>
> ruiyun wan <wr...@gmail.com> 于2022年4月26日周二 14:41写道:
>
> > Flink版本:1.13
> > 问题描述:使用sql-client.sh启动yarn-per-job(execution.target =
> >
> >
> yarn-per-job)时,在YARN集群侧生成的launch_container.sh中,启动org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint的参数中无-Dlog.file和-Dlog4j.configuration属性参数,导致没有jobmanager.log日志文件,如何设置才能影响Yarn生成的launch_container.sh包含上述参数。
> >
>

Re: 关于使用SQL Client提交执行目标为yarn-per-job的作业时无日志文件(jobmanager.log)的求助

Posted by Qishang <zh...@gmail.com>.
Hi.
确认下 conf 下,是否有 log4j.properties

应该是在这个地放生成的,
https://github.com/apache/flink/blob/release-1.13/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java#L1699


ruiyun wan <wr...@gmail.com> 于2022年4月26日周二 14:41写道:

> Flink版本:1.13
> 问题描述:使用sql-client.sh启动yarn-per-job(execution.target =
>
> yarn-per-job)时,在YARN集群侧生成的launch_container.sh中,启动org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint的参数中无-Dlog.file和-Dlog4j.configuration属性参数,导致没有jobmanager.log日志文件,如何设置才能影响Yarn生成的launch_container.sh包含上述参数。
>