Changing the order of exec command makes sense to me. Would you please
create a ticket for this?

The /opt/flink/conf is cleaned up because we are mounting the conf files
from K8s ConfigMap.



Best,
Yang

Tamir Sagi <tamir.s...@niceactimize.com> 于2022年1月18日周二 17:48写道:

> Hey Yang,
>
> Thank you for confirming it.
>
> IMO, a better approach is to change the order "log_setting" , "ARGS" and 
> "FLINK_ENV_JAVA_OPTS"
> in exec command.
> In that way we prioritize user defined properties.
>
> From:
>
> exec "$JAVA_RUN" $JVM_ARGS ${FLINK_ENV_JAVA_OPTS} "${log_setting[@]}"
> -classpath "`manglePathList
> "$FLINK_TM_CLASSPATH:$INTERNAL_HADOOP_CLASSPATHS"`" ${CLASS_TO_RUN}
> "${ARGS[@]}"
>
> To
>
> exec "$JAVA_RUN" $JVM_ARGS "${log_setting[@]}" -classpath "`manglePathList
> "$FLINK_TM_CLASSPATH:$INTERNAL_HADOOP_CLASSPATHS"`" ${CLASS_TO_RUN}
> "${ARGS[@]}" "${FLINK_ENV_JAVA_OPTS}"
>
> Unless there are system configurations which not supposed to be overridden
> by user(And then having dedicated env variables is better). does it make
> sense to you?
>
>
> In addition, any idea why /opt/flink/conf gets cleaned (Only
> flink-conf.xml is there).
>
>
> Best,
> Tamir
>
>
> ------------------------------
> *From:* Yang Wang <danrtsey...@gmail.com>
> *Sent:* Tuesday, January 18, 2022 6:02 AM
> *To:* Tamir Sagi <tamir.s...@niceactimize.com>
> *Cc:* user@flink.apache.org <user@flink.apache.org>
> *Subject:* Re: Flink 1.14.2 - Log4j2 -Dlog4j.configurationFile is ignored
> and falls back to default /opt/flink/conf/log4j-console.properties
>
>
> *EXTERNAL EMAIL*
>
>
> I think you are right. Before 1.13.0, if the log configuration file does
> not exist, the logging properties would not be added to the start command.
> That is why it could work in 1.12.2.
>
> However, from 1.13.0, we are not using
> "kubernetes.container-start-command-template" to generate the JM/TM start
> command, but the jobmanager.sh/taskmanager.sh. We do not
> have the same logic in the "flink-console.sh".
>
> Maybe we could introduce an environment for log configuration file name in
> the "flink-console.sh". The default value could be
> "log4j-console.properties" and it could be configured by users.
> If this makes sense to you, could you please create a ticket?
>
>
> Best,
> Yang
>
> Tamir Sagi <tamir.s...@niceactimize.com> 于2022年1月17日周一 22:53写道:
>
> Hey Yang,
>
> thanks for answering,
>
> TL;DR
>
> Assuming I have not missed anything , the way TM and JM are created is
> different between these 2 versions,
> but it does look like flink-console.sh gets called eventually with the
> same exec command.
>
> in 1.12.2 if org.apache.flink.kubernetes.kubeclient.parameters#hasLog4j
> returns false then logging args are not added to startCommand.
>
>
>    1. why does the config dir gets cleaned once the cluster starts? Even
>    when I pushed log4j-console.properties to the expected location
>    (/opt/flink/conf) , the directory includes only flink-conf.yaml.
>    2. I think by running exec command "...${FLINK_ENV_JAVA_OPTS}
>    "${log_setting[@]}" ...."${ARGS[@]}" some properties might be ignored.
>    IMO, it should first look for properties in java.opts provided by the
>    user in flink-conf and falls back to default in case it's not present.
>
>
> Taking about Native kubernetes mode
>
> I checked the bash script in flink-dist module, it looks like in both
> 1.14.2 and 1.12.2. flink-console.sh is similar. (in 1.14.2 there are more
> cases for the input argument)
>
> logging variable is the same
>
> https://github.com/apache/flink/blob/release-1.14.2/flink-dist/src/main/flink-bin/bin/flink-console.sh#L101
>
> https://github.com/apache/flink/blob/release-1.12.2/flink-dist/src/main/flink-bin/bin/flink-console.sh#L89
>
> Exec command is the same
>
> https://github.com/apache/flink/blob/release-1.14.2/flink-dist/src/main/flink-bin/bin/flink-console.sh#L114
>
> https://github.com/apache/flink/blob/release-1.12.2/flink-dist/src/main/flink-bin/bin/flink-console.sh#L99
>
> As for creating TM/JM, in *1.14.2* there is a usage of 2 bash scripts
>
>    - kubernetes-jobmanager.sh
>    - kubernetes-taskmanager.sh
>
> They get called while decorating the pod, referenced in startCommand.
>
> for instance, JobManager.
>
> https://github.com/apache/flink/blob/release-1.14.2/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/CmdJobManagerDecorator.java#L58-L59
>
> kubernetes-jobmanager.sh gets called once the container starts which calls
> flink-console.sh internally and pass the
> deploymentName(kubernetes-application in our case) and args.
>
> In *1.12.2* the decorator set /docker-entrypoint.sh
>
> https://github.com/apache/flink/blob/release-1.12.2/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactory.java#L67
>
> and set the start command
>
> https://github.com/apache/flink/blob/release-1.12.2/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java#L224
>
>
> https://github.com/apache/flink/blob/release-1.12.2/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java#L333
>
>
> with additional logging parameter
>
> https://github.com/apache/flink/blob/release-1.12.2/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
> #L421-L425
> <https://github.com/apache/flink/blob/4dedee047bc69d219095bd98782c6e95f04a6cb9/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java#L421-L425>
>
> hasLog4j
>
> https://github.com/apache/flink/blob/release-1.12.2/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/AbstractKubernetesParameters.java#L151-L155
> it checks if the file exists in conf dir.
>
> If the log4j is false, then the logging properties are not added to start
> command(Might be the case, which explains why it works in 1.12.2)
>
> It then passes 'jobmanager' as component.
> looking into /docker-entrypoint.sh it calls jobmanager.sh which calls
> flink-console.sh internally
>
> Have I missed anything?
>
>
> Best,
> Tamir
>
>
> ------------------------------
> *From:* Yang Wang <danrtsey...@gmail.com>
> *Sent:* Monday, January 17, 2022 1:05 PM
> *To:* Tamir Sagi <tamir.s...@niceactimize.com>
> *Cc:* user@flink.apache.org <user@flink.apache.org>
> *Subject:* Re: Flink 1.14.2 - Log4j2 -Dlog4j.configurationFile is ignored
> and falls back to default /opt/flink/conf/log4j-console.properties
>
>
> *EXTERNAL EMAIL*
>
>
> I think the root cause is that we are using "flink-console.sh" to start
> the JobManager/TaskManager process for native K8s integration after
> FLINK-21128[1].
> So it forces the log4j configuration name to be "log4j-console.properties".
>
>
> [1]. https://issues.apache.org/jira/browse/FLINK-21128
>
>
> Best,
> Yang
>
> Tamir Sagi <tamir.s...@niceactimize.com> 于2022年1月13日周四 20:30写道:
>
> Hey All
>
> I'm Running Flink 1.14.2, it seems like it ignores system
> property -Dlog4j.configurationFile and
> falls back to /opt/flink/conf/log4j-console.properties
>
> I enabled debug log for log4j2  ( -Dlog4j2.debug)
>
> DEBUG StatusLogger Catching
>  java.io.FileNotFoundException:
> file:/opt/flink/conf/log4j-console.properties (No such file or directory)
> at java.base/java.io.FileInputStream.open0(Native Method)
> at java.base/java.io.FileInputStream.open(Unknown Source)
> at java.base/java.io.FileInputStream.<init>(Unknown Source)
> at
> org.apache.logging.log4j.core.config.ConfigurationFactory.getInputFromString(ConfigurationFactory.java:370)
> at
> org.apache.logging.log4j.core.config.ConfigurationFactory$Factory.getConfiguration(ConfigurationFactory.java:513)
> at
> org.apache.logging.log4j.core.config.ConfigurationFactory$Factory.getConfiguration(ConfigurationFactory.java:499)
> at
> org.apache.logging.log4j.core.config.ConfigurationFactory$Factory.getConfiguration(ConfigurationFactory.java:422)
> at
> org.apache.logging.log4j.core.config.ConfigurationFactory.getConfiguration(ConfigurationFactory.java:322)
> at
> org.apache.logging.log4j.core.LoggerContext.reconfigure(LoggerContext.java:695)
> at
> org.apache.logging.log4j.core.LoggerContext.reconfigure(LoggerContext.java:716)
> at
> org.apache.logging.log4j.core.LoggerContext.start(LoggerContext.java:270)
> at
> org.apache.logging.log4j.core.impl.Log4jContextFactory.getContext(Log4jContextFactory.java:155)
> at
> org.apache.logging.log4j.core.impl.Log4jContextFactory.getContext(Log4jContextFactory.java:47)
> at org.apache.logging.log4j.LogManager.getContext(LogManager.java:196)
> at
> org.apache.logging.log4j.spi.AbstractLoggerAdapter.getContext(AbstractLoggerAdapter.java:137)
> at
> org.apache.logging.slf4j.Log4jLoggerFactory.getContext(Log4jLoggerFactory.java:55)
> at
> org.apache.logging.log4j.spi.AbstractLoggerAdapter.getLogger(AbstractLoggerAdapter.java:47)
> at
> org.apache.logging.slf4j.Log4jLoggerFactory.getLogger(Log4jLoggerFactory.java:33)
> at org.slf4j.LoggerFactory.getLogger(LoggerFactory.java:329)
> at org.slf4j.LoggerFactory.getLogger(LoggerFactory.java:349)
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils.<clinit>(AkkaRpcServiceUtils.java:55)
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcSystem.remoteServiceBuilder(AkkaRpcSystem.java:42)
> at
> org.apache.flink.runtime.rpc.akka.CleanupOnCloseRpcSystem.remoteServiceBuilder(CleanupOnCloseRpcSystem.java:77)
> at
> org.apache.flink.runtime.rpc.RpcUtils.createRemoteRpcService(RpcUtils.java:184)
> at
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.initializeServices(ClusterEntrypoint.java:300)
> at
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.runCluster(ClusterEntrypoint.java:243)
> at
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.lambda$startCluster$1(ClusterEntrypoint.java:193)
> at
> org.apache.flink.runtime.security.contexts.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:28)
> at
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.startCluster(ClusterEntrypoint.java:190)
> at
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.runClusterEntrypoint(ClusterEntrypoint.java:617)
>
> Where I see the property is being loaded while deploying the cluster
>
> source:{
> class:org.apache.flink.configuration.GlobalConfiguration
> method:loadYAMLResource
> file:GlobalConfiguration.java
> line:213
> }
> message:Loading configuration property: env.java.opts,
> -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/dumps
> -Dlog4j.configurationFile=/opt/log4j2/log4j2.xml -Dlog4j2.debug=true
>
> in addition,  following the documentation[1], it seems like Flink comes
> with default log4j properties files located in /opt/flink/conf
>
> looking into that dir once the cluster is deployed, only flink-conf.yaml
> is there.
>
>
>
> Docker file content
>
> FROM flink:1.14.2-scala_2.12-java11
> ARG JAR_FILE
> COPY target/${JAR_FILE} $FLINK_HOME/usrlib/flink-job.jar
> ADD log4j2.xml /opt/log4j2/log4j2.xml
>
>
>
> *It perfectly works in 1.12.2 with the same log4j2.xml file and system
> property. *
>
> [1]
> https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/advanced/logging/#configuring-log4j-2
>
>
> Best,
> Tamir
>
>
>
> Confidentiality: This communication and any attachments are intended for
> the above-named persons only and may be confidential and/or legally
> privileged. Any opinions expressed in this communication are not
> necessarily those of NICE Actimize. If this communication has come to you
> in error you must take no action based on it, nor must you copy or show it
> to anyone; please delete/destroy and inform the sender by e-mail
> immediately.
> Monitoring: NICE Actimize may monitor incoming and outgoing e-mails.
> Viruses: Although we have taken steps toward ensuring that this e-mail and
> attachments are free from any virus, we advise that in keeping with good
> computing practice the recipient should ensure they are actually virus free.
>
>
> Confidentiality: This communication and any attachments are intended for
> the above-named persons only and may be confidential and/or legally
> privileged. Any opinions expressed in this communication are not
> necessarily those of NICE Actimize. If this communication has come to you
> in error you must take no action based on it, nor must you copy or show it
> to anyone; please delete/destroy and inform the sender by e-mail
> immediately.
> Monitoring: NICE Actimize may monitor incoming and outgoing e-mails.
> Viruses: Although we have taken steps toward ensuring that this e-mail and
> attachments are free from any virus, we advise that in keeping with good
> computing practice the recipient should ensure they are actually virus free.
>
>
> Confidentiality: This communication and any attachments are intended for
> the above-named persons only and may be confidential and/or legally
> privileged. Any opinions expressed in this communication are not
> necessarily those of NICE Actimize. If this communication has come to you
> in error you must take no action based on it, nor must you copy or show it
> to anyone; please delete/destroy and inform the sender by e-mail
> immediately.
> Monitoring: NICE Actimize may monitor incoming and outgoing e-mails.
> Viruses: Although we have taken steps toward ensuring that this e-mail and
> attachments are free from any virus, we advise that in keeping with good
> computing practice the recipient should ensure they are actually virus free.
>

Reply via email to