[ https://issues.apache.org/jira/browse/FLINK-20143?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17231369#comment-17231369 ]
zhisheng commented on FLINK-20143: ---------------------------------- {code:java} 22020-11-13 18:46:43,014 INFO org.apache.flink.client.cli.CliFrontend [] - --------------------------------------------------------------------------------2020-11-13 18:46:43,014 INFO org.apache.flink.client.cli.CliFrontend [] - --------------------------------------------------------------------------------2020-11-13 18:46:43,019 INFO org.apache.flink.client.cli.CliFrontend [] - Starting Command Line Client (Version: 1.12-SNAPSHOT, Scala: 2.11, Rev:c55420b, Date:2020-11-05T05:29:49+01:00)2020-11-13 18:46:43,019 INFO org.apache.flink.client.cli.CliFrontend [] - OS current user: deploy2020-11-13 18:46:43,415 INFO org.apache.flink.client.cli.CliFrontend [] - Current Hadoop/Kerberos user: deploy2020-11-13 18:46:43,416 INFO org.apache.flink.client.cli.CliFrontend [] - JVM: Java HotSpot(TM) 64-Bit Server VM - Oracle Corporation - 1.8/25.92-b142020-11-13 18:46:43,416 INFO org.apache.flink.client.cli.CliFrontend [] - Maximum heap size: 7136 MiBytes2020-11-13 18:46:43,416 INFO org.apache.flink.client.cli.CliFrontend [] - JAVA_HOME: /app/jdk/2020-11-13 18:46:43,418 INFO org.apache.flink.client.cli.CliFrontend [] - Hadoop version: 2.7.32020-11-13 18:46:43,418 INFO org.apache.flink.client.cli.CliFrontend [] - JVM Options:2020-11-13 18:46:43,418 INFO org.apache.flink.client.cli.CliFrontend [] - -Dlog.file=/data1/app/flink-1.12-SNAPSHOT/log/flink-deploy-client-FAT-hadoopuat-69120.vm.dc01. .tech.log2020-11-13 18:46:43,418 INFO org.apache.flink.client.cli.CliFrontend [] - -Dlog4j.configuration=file:/data1/app/flink-1.12-SNAPSHOT/conf/log4j-cli.properties2020-11-13 18:46:43,418 INFO org.apache.flink.client.cli.CliFrontend [] - -Dlog4j.configurationFile=file:/data1/app/flink-1.12-SNAPSHOT/conf/log4j-cli.properties2020-11-13 18:46:43,418 INFO org.apache.flink.client.cli.CliFrontend [] - -Dlogback.configurationFile=file:/data1/app/flink-1.12-SNAPSHOT/conf/logback.xml2020-11-13 18:46:43,419 INFO org.apache.flink.client.cli.CliFrontend [] - Program Arguments:2020-11-13 18:46:43,420 INFO org.apache.flink.client.cli.CliFrontend [] - run2020-11-13 18:46:43,420 INFO org.apache.flink.client.cli.CliFrontend [] - -t2020-11-13 18:46:43,421 INFO org.apache.flink.client.cli.CliFrontend [] - yarn-per-job2020-11-13 18:46:43,421 INFO org.apache.flink.client.cli.CliFrontend [] - -Dexecution.attached=false2020-11-13 18:46:43,421 INFO org.apache.flink.client.cli.CliFrontend [] - -Dyarn.provided.lib.dirs=hdfs:///flink/flink-1.12-SNAPSHOT/lib2020-11-13 18:46:43,421 INFO org.apache.flink.client.cli.CliFrontend [] - ./examples/streaming/StateMachineExample.jar2020-11-13 18:46:43,421 INFO org.apache.flink.client.cli.CliFrontend [] - Classpath: /data1/app/flink-1.12-SNAPSHOT/lib/flink-connector-jdbc_2.11-1.12-SNAPSHOT.jar:/data1/app/flink-1.12-SNAPSHOT/lib/flink-csv-1.12-SNAPSHOT.jar:/data1/app/flink-1.12-SNAPSHOT/lib/flink-json-1.12-SNAPSHOT.jar:/data1/app/flink-1.12-SNAPSHOT/lib/flink-shaded-zookeeper-3.4.14.jar:/data1/app/flink-1.12-SNAPSHOT/lib/flink-sql-connector-elasticsearch7_2.11-1.12-SNAPSHOT.jar:/data1/app/flink-1.12-SNAPSHOT/lib/flink-sql-connector-kafka_2.11-1.12-SNAPSHOT.jar:/data1/app/flink-1.12-SNAPSHOT/lib/flink-table_2.11-1.12-SNAPSHOT.jar:/data1/app/flink-1.12-SNAPSHOT/lib/flink-table-blink_2.11-1.12-SNAPSHOT.jar:/data1/app/flink-1.12-SNAPSHOT/lib/log4j-1.2-api-2.12.1.jar:/data1/app/flink-1.12-SNAPSHOT/lib/log4j-api-2.12.1.jar:/data1/app/flink-1.12-SNAPSHOT/lib/log4j-core-2.12.1.jar:/data1/app/flink-1.12-SNAPSHOT/lib/log4j-slf4j-impl-2.12.1.jar:/data1/app/flink-1.12-SNAPSHOT/lib/flink-dist_2.11-1.12-SNAPSHOT.jar:/app/hadoop/etc/hadoop:/app/hadoop/share/hadoop/common/lib/log4j-1.2.17.jar:/app/hadoop/share/hadoop/common/lib/jsr305-3.0.0.jar:/app/hadoop/share/hadoop/common/lib/jackson-core-asl-1.9.13.jar:/app/hadoop/share/hadoop/common/lib/commons-httpclient-3.1.jar:/app/hadoop/share/hadoop/common/lib/jackson-xc-1.9.13.jar:/app/hadoop/share/hadoop/common/lib/servlet-api-2.5.jar:/app/hadoop/share/hadoop/common/lib/httpcore-4.2.5.jar:/app/hadoop/share/hadoop/common/lib/jettison-1.1.jar:/app/hadoop/share/hadoop/common/lib/stax-api-1.0-2.jar:/app/hadoop/share/hadoop/common/lib/jsch-0.1.42.jar:/app/hadoop/share/hadoop/common/lib/httpclient-4.2.5.jar:/app/hadoop/share/hadoop/common/lib/apacheds-kerberos-codec-2.0.0-M15.jar:/app/hadoop/share/hadoop/common/lib/htrace-core-3.1.0-incubating.jar:/app/hadoop/share/hadoop/common/lib/commons-cli-1.2.jar:/app/hadoop/share/hadoop/common/lib/jersey-server-1.9.jar:/app/hadoop/share/hadoop/common/lib/xz-1.0.jar:/app/hadoop/share/hadoop/common/lib/jackson-mapper-asl-1.9.13.jar:/app/hadoop/share/hadoop/common/lib/snappy-java-1.0.4.1.jar:/app/hadoop/share/hadoop/common/lib/curator-recipes-2.7.1.jar:/app/hadoop/share/hadoop/common/lib/gson-2.2.4.jar:/app/hadoop/share/hadoop/common/lib/hadoop-annotations-2.7.3.jar:/app/hadoop/share/hadoop/common/lib/java-xmlbuilder-0.4.jar:/app/hadoop/share/hadoop/common/lib/jersey-core-1.9.jar:/app/hadoop/share/hadoop/common/lib/apacheds-i18n-2.0.0-M15.jar:/app/hadoop/share/hadoop/common/lib/commons-collections-3.2.2.jar:/app/hadoop/share/hadoop/common/lib/jackson-jaxrs-1.9.13.jar:/app/hadoop/share/hadoop/common/lib/api-util-1.0.0-M20.jar:/app/hadoop/share/hadoop/common/lib/slf4j-log4j12-1.7.10.jar:/app/hadoop/share/hadoop/common/lib/commons-codec-1.4.jar:/app/hadoop/share/hadoop/common/lib/hellobike-rack-awareness-policy-1.0.jar:/app/hadoop/share/hadoop/common/lib/paranamer-2.3.jar:/app/hadoop/share/hadoop/common/lib/jetty-util-6.1.26.jar:/app/hadoop/share/hadoop/common/lib/commons-digester-1.8.jar:/app/hadoop/share/hadoop/common/lib/curator-framework-2.7.1.jar:/app/hadoop/share/hadoop/common/lib/zookeeper-3.4.6.jar:/app/hadoop/share/hadoop/common/lib/commons-net-3.1.jar:/app/hadoop/share/hadoop/common/lib/commons-math3-3.1.1.jar:/app/hadoop/share/hadoop/common/lib/protobuf-java-2.5.0.jar:/app/hadoop/share/hadoop/common/lib/hadoop-auth-2.7.3.jar:/app/hadoop/share/hadoop/common/lib/guava-11.0.2.jar:/app/hadoop/share/hadoop/common/lib/jsp-api-2.1.jar:/app/hadoop/share/hadoop/common/lib/commons-beanutils-core-1.8.0.jar:/app/hadoop/share/hadoop/common/lib/jersey-json-1.9.jar:/app/hadoop/share/hadoop/common/lib/netty-3.6.2.Final.jar:/app/hadoop/share/hadoop/common/lib/xmlenc-0.52.jar:/app/hadoop/share/hadoop/common/lib/jets3t-0.9.0.jar:/app/hadoop/share/hadoop/common/lib/commons-configuration-1.6.jar:/app/hadoop/share/hadoop/common/lib/hamcrest-core-1.3.jar:/app/hadoop/share/hadoop/common/lib/avro-1.7.4.jar:/app/hadoop/share/hadoop/common/lib/commons-compress-1.4.1.jar:/app/hadoop/share/hadoop/common/lib/jetty-6.1.26.jar:/app/hadoop/share/hadoop/common/lib/junit-4.11.jar:/app/hadoop/share/hadoop/common/lib/api-asn1-api-1.0.0-M20.jar:/app/hadoop/share/hadoop/common/lib/commons-logging-1.1.3.jar:/app/hadoop/share/hadoop/common/lib/activation-1.1.jar:/app/hadoop/share/hadoop/common/lib/slf4j-api-1.7.10.jar:/app/hadoop/share/hadoop/common/lib/jaxb-api-2.2.2.jar:/app/hadoop/share/hadoop/common/lib/jaxb-impl-2.2.3-1.jar:/app/hadoop/share/hadoop/common/lib/curator-client-2.7.1.jar:/app/hadoop/share/hadoop/common/lib/mockito-all-1.8.5.jar:/app/hadoop/share/hadoop/common/lib/commons-beanutils-1.7.0.jar:/app/hadoop/share/hadoop/common/lib/commons-io-2.4.jar:/app/hadoop/share/hadoop/common/lib/commons-lang-2.6.jar:/app/hadoop/share/hadoop/common/lib/asm-3.2.jar:/app/hadoop/share/hadoop/common/hadoop-nfs-2.7.3.jar:/app/hadoop/share/hadoop/common/hadoop-common-2.7.3.jar:/app/hadoop/share/hadoop/common/hadoop-common-2.7.3-tests.jar:/app/hadoop/share/hadoop/hdfs:/app/hadoop/share/hadoop/hdfs/lib/log4j-1.2.17.jar:/app/hadoop/share/hadoop/hdfs/lib/jsr305-3.0.0.jar:/app/hadoop/share/hadoop/hdfs/lib/jackson-core-asl-1.9.13.jar:/app/hadoop/share/hadoop/hdfs/lib/servlet-api-2.5.jar:/app/hadoop/share/hadoop/hdfs/lib/ranger-plugin-classloader-1.1.0.jar:/app/hadoop/share/hadoop/hdfs/lib/ranger-hdfs-plugin-shim-1.1.0.jar:/app/hadoop/share/hadoop/hdfs/lib/htrace-core-3.1.0-incubating.jar:/app/hadoop/share/hadoop/hdfs/lib/commons-cli-1.2.jar:/app/hadoop/share/hadoop/hdfs/lib/jersey-server-1.9.jar:/app/hadoop/share/hadoop/hdfs/lib/jackson-mapper-asl-1.9.13.jar:/app/hadoop/share/hadoop/hdfs/lib/hellobike-block-placement-policy-1.0.jar:/app/hadoop/share/hadoop/hdfs/lib/jersey-core-1.9.jar:/app/hadoop/share/hadoop/hdfs/lib/commons-codec-1.4.jar:/app/hadoop/share/hadoop/hdfs/lib/leveldbjni-all-1.8.jar:/app/hadoop/share/hadoop/hdfs/lib/jetty-util-6.1.26.jar:/app/hadoop/share/hadoop/hdfs/lib/protobuf-java-2.5.0.jar:/app/hadoop/share/hadoop/hdfs/lib/xml-apis-1.3.04.jar:/app/hadoop/share/hadoop/hdfs/lib/guava-11.0.2.jar:/app/hadoop/share/hadoop/hdfs/lib/xercesImpl-2.9.1.jar:/app/hadoop/share/hadoop/hdfs/lib/netty-3.6.2.Final.jar:/app/hadoop/share/hadoop/hdfs/lib/xmlenc-0.52.jar:/app/hadoop/share/hadoop/hdfs/lib/jetty-6.1.26.jar:/app/hadoop/share/hadoop/hdfs/lib/commons-daemon-1.0.13.jar:/app/hadoop/share/hadoop/hdfs/lib/commons-logging-1.1.3.jar:/app/hadoop/share/hadoop/hdfs/lib/netty-all-4.0.23.Final.jar:/app/hadoop/share/hadoop/hdfs/lib/commons-io-2.4.jar:/app/hadoop/share/hadoop/hdfs/lib/commons-lang-2.6.jar:/app/hadoop/share/hadoop/hdfs/lib/asm-3.2.jar:/app/hadoop/share/hadoop/hdfs/hadoop-hdfs-2.7.3-tests.jar:/app/hadoop/share/hadoop/hdfs/hadoop-hdfs-nfs-2.7.3.jar:/app/hadoop/share/hadoop/hdfs/hadoop-hdfs-2.7.3.jar:/app/hadoop/share/hadoop/yarn/lib/log4j-1.2.17.jar:/app/hadoop/share/hadoop/yarn/lib/jsr305-3.0.0.jar:/app/hadoop/share/hadoop/yarn/lib/jackson-core-asl-1.9.13.jar:/app/hadoop/share/hadoop/yarn/lib/jackson-xc-1.9.13.jar:/app/hadoop/share/hadoop/yarn/lib/servlet-api-2.5.jar:/app/hadoop/share/hadoop/yarn/lib/jettison-1.1.jar:/app/hadoop/share/hadoop/yarn/lib/stax-api-1.0-2.jar:/app/hadoop/share/hadoop/yarn/lib/commons-cli-1.2.jar:/app/hadoop/share/hadoop/yarn/lib/jersey-server-1.9.jar:/app/hadoop/share/hadoop/yarn/lib/xz-1.0.jar:/app/hadoop/share/hadoop/yarn/lib/jackson-mapper-asl-1.9.13.jar:/app/hadoop/share/hadoop/yarn/lib/guice-3.0.jar:/app/hadoop/share/hadoop/yarn/lib/jersey-core-1.9.jar:/app/hadoop/share/hadoop/yarn/lib/commons-collections-3.2.2.jar:/app/hadoop/share/hadoop/yarn/lib/jackson-jaxrs-1.9.13.jar:/app/hadoop/share/hadoop/yarn/lib/jersey-guice-1.9.jar:/app/hadoop/share/hadoop/yarn/lib/commons-codec-1.4.jar:/app/hadoop/share/hadoop/yarn/lib/leveldbjni-all-1.8.jar:/app/hadoop/share/hadoop/yarn/lib/jetty-util-6.1.26.jar:/app/hadoop/share/hadoop/yarn/lib/zookeeper-3.4.6.jar:/app/hadoop/share/hadoop/yarn/lib/protobuf-java-2.5.0.jar:/app/hadoop/share/hadoop/yarn/lib/guava-11.0.2.jar:/app/hadoop/share/hadoop/yarn/lib/jersey-json-1.9.jar:/app/hadoop/share/hadoop/yarn/lib/spark-2.3.1-yarn-shuffle.jar:/app/hadoop/share/hadoop/yarn/lib/netty-3.6.2.Final.jar:/app/hadoop/share/hadoop/yarn/lib/aopalliance-1.0.jar:/app/hadoop/share/hadoop/yarn/lib/guice-servlet-3.0.jar:/app/hadoop/share/hadoop/yarn/lib/javax.inject-1.jar:/app/hadoop/share/hadoop/yarn/lib/zookeeper-3.4.6-tests.jar:/app/hadoop/share/hadoop/yarn/lib/commons-compress-1.4.1.jar:/app/hadoop/share/hadoop/yarn/lib/jetty-6.1.26.jar:/app/hadoop/share/hadoop/yarn/lib/commons-logging-1.1.3.jar:/app/hadoop/share/hadoop/yarn/lib/activation-1.1.jar:/app/hadoop/share/hadoop/yarn/lib/jaxb-api-2.2.2.jar:/app/hadoop/share/hadoop/yarn/lib/jaxb-impl-2.2.3-1.jar:/app/hadoop/share/hadoop/yarn/lib/jersey-client-1.9.jar:/app/hadoop/share/hadoop/yarn/lib/commons-io-2.4.jar:/app/hadoop/share/hadoop/yarn/lib/commons-lang-2.6.jar:/app/hadoop/share/hadoop/yarn/lib/asm-3.2.jar:/app/hadoop/share/hadoop/yarn/hadoop-yarn-registry-2.7.3.jar:/app/hadoop/share/hadoop/yarn/hadoop-yarn-client-2.7.3.jar:/app/hadoop/share/hadoop/yarn/hadoop-yarn-api-2.7.3.jar:/app/hadoop/share/hadoop/yarn/hadoop-yarn-applications-distributedshell-2.7.3.jar:/app/hadoop/share/hadoop/yarn/hadoop-yarn-server-sharedcachemanager-2.7.3.jar:/app/hadoop/share/hadoop/yarn/hadoop-yarn-server-applicationhistoryservice-2.7.3.jar:/app/hadoop/share/hadoop/yarn/hadoop-yarn-server-resourcemanager-2.7.3.jar:/app/hadoop/share/hadoop/yarn/hadoop-yarn-server-nodemanager-2.7.3.jar:/app/hadoop/share/hadoop/yarn/hadoop-yarn-common-2.7.3.jar:/app/hadoop/share/hadoop/yarn/hadoop-yarn-server-tests-2.7.3.jar:/app/hadoop/share/hadoop/yarn/hadoop-yarn-server-web-proxy-2.7.3.jar:/app/hadoop/share/hadoop/yarn/hadoop-yarn-server-common-2.7.3.jar:/app/hadoop/share/hadoop/yarn/hadoop-yarn-applications-unmanaged-am-launcher-2.7.3.jar:/app/hadoop/share/hadoop/mapreduce/lib/log4j-1.2.17.jar:/app/hadoop/share/hadoop/mapreduce/lib/jackson-core-asl-1.9.13.jar:/app/hadoop/share/hadoop/mapreduce/lib/jersey-server-1.9.jar:/app/hadoop/share/hadoop/mapreduce/lib/xz-1.0.jar:/app/hadoop/share/hadoop/mapreduce/lib/jackson-mapper-asl-1.9.13.jar:/app/hadoop/share/hadoop/mapreduce/lib/snappy-java-1.0.4.1.jar:/app/hadoop/share/hadoop/mapreduce/lib/guice-3.0.jar:/app/hadoop/share/hadoop/mapreduce/lib/hadoop-annotations-2.7.3.jar:/app/hadoop/share/hadoop/mapreduce/lib/jersey-core-1.9.jar:/app/hadoop/share/hadoop/mapreduce/lib/jersey-guice-1.9.jar:/app/hadoop/share/hadoop/mapreduce/lib/leveldbjni-all-1.8.jar:/app/hadoop/share/hadoop/mapreduce/lib/paranamer-2.3.jar:/app/hadoop/share/hadoop/mapreduce/lib/protobuf-java-2.5.0.jar:/app/hadoop/share/hadoop/mapreduce/lib/netty-3.6.2.Final.jar:/app/hadoop/share/hadoop/mapreduce/lib/aopalliance-1.0.jar:/app/hadoop/share/hadoop/mapreduce/lib/guice-servlet-3.0.jar:/app/hadoop/share/hadoop/mapreduce/lib/javax.inject-1.jar:/app/hadoop/share/hadoop/mapreduce/lib/hamcrest-core-1.3.jar:/app/hadoop/share/hadoop/mapreduce/lib/avro-1.7.4.jar:/app/hadoop/share/hadoop/mapreduce/lib/commons-compress-1.4.1.jar:/app/hadoop/share/hadoop/mapreduce/lib/junit-4.11.jar:/app/hadoop/share/hadoop/mapreduce/lib/commons-io-2.4.jar:/app/hadoop/share/hadoop/mapreduce/lib/asm-3.2.jar:/app/hadoop/share/hadoop/mapreduce/hadoop-mapreduce-client-jobclient-2.7.3-tests.jar:/app/hadoop/share/hadoop/mapreduce/hadoop-mapreduce-client-common-2.7.3.jar:/app/hadoop/share/hadoop/mapreduce/hadoop-mapreduce-client-hs-2.7.3.jar:/app/hadoop/share/hadoop/mapreduce/hadoop-mapreduce-client-core-2.7.3.jar:/app/hadoop/share/hadoop/mapreduce/hadoop-mapreduce-client-app-2.7.3.jar:/app/hadoop/share/hadoop/mapreduce/hadoop-mapreduce-examples-2.7.3.jar:/app/hadoop/share/hadoop/mapreduce/hadoop-mapreduce-client-shuffle-2.7.3.jar:/app/hadoop/share/hadoop/mapreduce/hadoop-mapreduce-client-jobclient-2.7.3.jar:/app/hadoop/share/hadoop/mapreduce/hadoop-mapreduce-client-hs-plugins-2.7.3.jar:/app/hadoop/share/hadoop/tools/lib/hadoop-aliyun-2.9.2-jar-with-dependencies.jar:/app/hadoop/contrib/capacity-scheduler/*.jar:/app/hadoop/etc/hadoop:/app/hadoop/etc/hadoop:/app/hbase/conf2020-11-13 18:46:43,422 INFO org.apache.flink.client.cli.CliFrontend [] - --------------------------------------------------------------------------------2020-11-13 18:46:43,426 INFO org.apache.flink.configuration.GlobalConfiguration [] - Loading configuration property: jobmanager.rpc.address, localhost2020-11-13 18:46:43,426 INFO org.apache.flink.configuration.GlobalConfiguration [] - Loading configuration property: jobmanager.rpc.port, 61232020-11-13 18:46:43,426 INFO org.apache.flink.configuration.GlobalConfiguration [] - Loading configuration property: jobmanager.memory.process.size, 1600m2020-11-13 18:46:43,426 INFO org.apache.flink.configuration.GlobalConfiguration [] - Loading configuration property: taskmanager.memory.process.size, 1728m2020-11-13 18:46:43,426 INFO org.apache.flink.configuration.GlobalConfiguration [] - Loading configuration property: taskmanager.numberOfTaskSlots, 22020-11-13 18:46:43,426 INFO org.apache.flink.configuration.GlobalConfiguration [] - Loading configuration property: parallelism.default, 12020-11-13 18:46:43,426 INFO org.apache.flink.configuration.GlobalConfiguration [] - Loading configuration property: high-availability, zookeeper2020-11-13 18:46:43,427 INFO org.apache.flink.configuration.GlobalConfiguration [] - Loading configuration property: high-availability.storageDir, hdfs:///flink/ha/2020-11-13 18:46:43,427 INFO org.apache.flink.configuration.GlobalConfiguration [] - Loading configuration property: high-availability.zookeeper.quorum, 10.69.1.15:2181,10.69.1.16:2181,10.69.1.17:21812020-11-13 18:46:43,427 INFO org.apache.flink.configuration.GlobalConfiguration [] - Loading configuration property: state.backend, rocksdb2020-11-13 18:46:43,427 INFO org.apache.flink.configuration.GlobalConfiguration [] - Loading configuration property: state.checkpoints.dir, hdfs:///flink/checkpoints2020-11-13 18:46:43,427 INFO org.apache.flink.configuration.GlobalConfiguration [] - Loading configuration property: state.savepoints.dir, hdfs:///flink/savepoints2020-11-13 18:46:43,427 INFO org.apache.flink.configuration.GlobalConfiguration [] - Loading configuration property: state.backend.incremental, true2020-11-13 18:46:43,427 INFO org.apache.flink.configuration.GlobalConfiguration [] - Loading configuration property: state.checkpoints.num-retained, 22020-11-13 18:46:43,427 INFO org.apache.flink.configuration.GlobalConfiguration [] - Loading configuration property: jobmanager.execution.failover-strategy, region2020-11-13 18:46:43,427 INFO org.apache.flink.configuration.GlobalConfiguration [] - Loading configuration property: execution.checkpointing.externalized-checkpoint-retention, RETAIN_ON_CANCELLATION2020-11-13 18:46:43,428 INFO org.apache.flink.configuration.GlobalConfiguration [] - Loading configuration property: execution.checkpointing.interval, 120s2020-11-13 18:46:43,428 INFO org.apache.flink.configuration.GlobalConfiguration [] - Loading configuration property: execution.checkpointing.mode, AT_LEAST_ONCE2020-11-13 18:46:43,428 INFO org.apache.flink.configuration.GlobalConfiguration [] - Loading configuration property: execution.checkpointing.timeout, 20 min2020-11-13 18:46:43,428 INFO org.apache.flink.configuration.GlobalConfiguration [] - Loading configuration property: execution.checkpointing.min-pause, 1 s2020-11-13 18:46:43,428 INFO org.apache.flink.configuration.GlobalConfiguration [] - Loading configuration property: historyserver.web.address, fat-hadoopuat-69120.vm.dc01. .tech2020-11-13 18:46:43,428 INFO org.apache.flink.configuration.GlobalConfiguration [] - Loading configuration property: historyserver.web.port, 80822020-11-13 18:46:43,428 INFO org.apache.flink.configuration.GlobalConfiguration [] - Loading configuration property: historyserver.archive.fs.refresh-interval, 100002020-11-13 18:46:43,429 INFO org.apache.flink.configuration.GlobalConfiguration [] - Loading configuration property: jobmanager.archive.fs.dir, hdfs:///flink/history-log2020-11-13 18:46:43,429 INFO org.apache.flink.configuration.GlobalConfiguration [] - Loading configuration property: historyserver.archive.fs.dir, hdfs:///flink/history-log2020-11-13 18:46:43,611 INFO org.apache.flink.runtime.security.modules.HadoopModule [] - Hadoop user set to deploy (auth:SIMPLE)2020-11-13 18:46:43,618 INFO org.apache.flink.runtime.security.modules.JaasModule [] - Jaas file will be created as /tmp/jaas-264858002832469329.conf.2020-11-13 18:46:43,627 INFO org.apache.flink.client.cli.CliFrontend [] - Running 'run' command.2020-11-13 18:46:43,648 INFO org.apache.flink.client.cli.CliFrontend [] - Building program from JAR file2020-11-13 18:46:43,666 INFO org.apache.flink.client.ClientUtils [] - Starting program (detached: false)2020-11-13 18:46:43,717 INFO org.apache.flink.contrib.streaming.state.RocksDBStateBackend [] - Using predefined options: DEFAULT.2020-11-13 18:46:43,718 INFO org.apache.flink.contrib.streaming.state.RocksDBStateBackend [] - Using default options factory: DefaultConfigurableOptionsFactory{configuredOptions={}}.2020-11-13 18:46:43,768 INFO org.apache.flink.api.java.typeutils.TypeExtractor [] - class org.apache.flink.streaming.examples.statemachine.event.Event does not contain a setter for field type2020-11-13 18:46:43,768 INFO org.apache.flink.api.java.typeutils.TypeExtractor [] - Class class org.apache.flink.streaming.examples.statemachine.event.Event cannot be used as a POJO type because not all fields are valid POJO fields, and must be processed as GenericType. Please read the Flink documentation on "Data Types & Serialization" for details of the effect on performance.2020-11-13 18:46:43,883 INFO org.apache.flink.api.java.typeutils.TypeExtractor [] - class org.apache.flink.streaming.examples.statemachine.event.Alert does not contain a setter for field address2020-11-13 18:46:43,883 INFO org.apache.flink.api.java.typeutils.TypeExtractor [] - Class class org.apache.flink.streaming.examples.statemachine.event.Alert cannot be used as a POJO type because not all fields are valid POJO fields, and must be processed as GenericType. Please read the Flink documentation on "Data Types & Serialization" for details of the effect on performance.2020-11-13 18:46:44,015 WARN org.apache.flink.yarn.configuration.YarnLogConfigUtil [] - The configuration directory ('/data1/app/flink-1.12-SNAPSHOT/conf') already contains a LOG4J config file.If you want to use logback, then please delete or rename the log configuration file.2020-11-13 18:46:44,228 INFO org.apache.hadoop.yarn.client.AHSProxy [] - Connecting to Application History server at FAT-hadoopuat-69117.vm.dc01. .tech/10.69.1.17:102002020-11-13 18:46:44,238 INFO org.apache.flink.yarn.YarnClusterDescriptor [] - No path for the flink jar passed. Using the location of class org.apache.flink.yarn.YarnClusterDescriptor to locate the jar2020-11-13 18:46:44,255 INFO org.apache.flink.runtime.util.config.memory.ProcessMemoryUtils [] - The derived from fraction jvm overhead memory (160.000mb (167772162 bytes)) is less than its min value 192.000mb (201326592 bytes), min value will be used instead2020-11-13 18:46:44,259 INFO org.apache.flink.runtime.util.config.memory.ProcessMemoryUtils [] - The derived from fraction jvm overhead memory (172.800mb (181193935 bytes)) is less than its min value 192.000mb (201326592 bytes), min value will be used instead2020-11-13 18:46:44,352 INFO org.apache.hadoop.yarn.client.ConfiguredRMFailoverProxyProvider [] - Failing over to rm22020-11-13 18:46:44,412 INFO org.apache.flink.yarn.YarnClusterDescriptor [] - The configured JobManager memory is 1600 MB. YARN will allocate 2048 MB to make up an integer multiple of its minimum allocation memory (2048 MB, configured via 'yarn.scheduler.minimum-allocation-mb'). The extra 448 MB may not be used by Flink.2020-11-13 18:46:44,413 INFO org.apache.flink.yarn.YarnClusterDescriptor [] - The configured TaskManager memory is 1728 MB. YARN will allocate 2048 MB to make up an integer multiple of its minimum allocation memory (2048 MB, configured via 'yarn.scheduler.minimum-allocation-mb'). The extra 320 MB may not be used by Flink.2020-11-13 18:46:44,413 INFO org.apache.flink.yarn.YarnClusterDescriptor [] - Cluster specification: ClusterSpecification{masterMemoryMB=2048, taskManagerMemoryMB=1728, slotsPerTaskManager=2}2020-11-13 18:46:44,960 WARN org.apache.hadoop.hdfs.shortcircuit.DomainSocketFactory [] - The short-circuit local reads feature cannot be used because libhadoop cannot be loaded.2020-11-13 18:46:47,129 INFO org.apache.flink.runtime.util.config.memory.ProcessMemoryUtils [] - The derived from fraction jvm overhead memory (160.000mb (167772162 bytes)) is less than its min value 192.000mb (201326592 bytes), min value will be used instead2020-11-13 18:46:47,146 INFO org.apache.flink.yarn.YarnClusterDescriptor [] - Submitting application master application_1599741232083_220232020-11-13 18:46:47,378 INFO org.apache.hadoop.yarn.client.api.impl.YarnClientImpl [] - Submitted application application_1599741232083_220232020-11-13 18:46:47,378 INFO org.apache.flink.yarn.YarnClusterDescriptor [] - Waiting for the cluster to be allocated2020-11-13 18:46:47,380 INFO org.apache.flink.yarn.YarnClusterDescriptor [] - Deploying cluster, current state ACCEPTED2020-11-13 18:46:52,167 ERROR org.apache.flink.client.cli.CliFrontend [] - Error while running the command.org.apache.flink.client.program.ProgramInvocationException: The main method caused an error: Could not deploy Yarn job cluster. at org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:330) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] at org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:198) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] at org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:114) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] at org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:743) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:242) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] at org.apache.flink.client.cli.CliFrontend.parseAndRun(CliFrontend.java:971) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] at org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:1047) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] at java.security.AccessController.doPrivileged(Native Method) ~[?:1.8.0_92] at javax.security.auth.Subject.doAs(Subject.java:422) [?:1.8.0_92] at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1698) [hadoop-common-2.7.3.jar:?] at org.apache.flink.runtime.security.contexts.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:41) [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:1047) [flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]Caused by: org.apache.flink.client.deployment.ClusterDeploymentException: Could not deploy Yarn job cluster. at org.apache.flink.yarn.YarnClusterDescriptor.deployJobCluster(YarnClusterDescriptor.java:460) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] at org.apache.flink.client.deployment.executors.AbstractJobClusterExecutor.execute(AbstractJobClusterExecutor.java:70) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] at org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.executeAsync(StreamExecutionEnvironment.java:1916) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] at org.apache.flink.client.program.StreamContextEnvironment.executeAsync(StreamContextEnvironment.java:128) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] at org.apache.flink.client.program.StreamContextEnvironment.execute(StreamContextEnvironment.java:76) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] at org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.java:1798) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] at org.apache.flink.streaming.examples.statemachine.StateMachineExample.main(StateMachineExample.java:142) ~[?:?] at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) ~[?:1.8.0_92] at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) ~[?:1.8.0_92] at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) ~[?:1.8.0_92] at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_92] at org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:316) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] ... 11 moreCaused by: org.apache.flink.yarn.YarnClusterDescriptor$YarnDeploymentException: The YARN application unexpectedly switched to state FAILED during deployment.Diagnostics from YARN: Application application_1599741232083_22023 failed 2 times in previous 10000 milliseconds due to AM Container for appattempt_1599741232083_22023_000002 exited with exitCode: -1Failing this attempt.Diagnostics: [2020-11-13 18:46:51.947]Destination must be relativeFor more detailed output, check the application tracking page: http://FAT-hadoopuat-69117.vm.dc01. .tech:8188/applicationhistory/app/application_1599741232083_22023 Then click on links to logs of each attempt.. Failing the application.If log aggregation is enabled on your cluster, use this command to further investigate the issue:yarn logs -applicationId application_1599741232083_22023 at org.apache.flink.yarn.YarnClusterDescriptor.startAppMaster(YarnClusterDescriptor.java:1078) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] at org.apache.flink.yarn.YarnClusterDescriptor.deployInternal(YarnClusterDescriptor.java:558) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] at org.apache.flink.yarn.YarnClusterDescriptor.deployJobCluster(YarnClusterDescriptor.java:453) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] at org.apache.flink.client.deployment.executors.AbstractJobClusterExecutor.execute(AbstractJobClusterExecutor.java:70) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] at org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.executeAsync(StreamExecutionEnvironment.java:1916) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] at org.apache.flink.client.program.StreamContextEnvironment.executeAsync(StreamContextEnvironment.java:128) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] at org.apache.flink.client.program.StreamContextEnvironment.execute(StreamContextEnvironment.java:76) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] at org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.java:1798) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] at org.apache.flink.streaming.examples.statemachine.StateMachineExample.main(StateMachineExample.java:142) ~[?:?] at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) ~[?:1.8.0_92] at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) ~[?:1.8.0_92] at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) ~[?:1.8.0_92] at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_92] at org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:316) ~[flink-dist_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT] ... 11 more2020-11-13 18:46:52,177 INFO org.apache.flink.yarn.YarnClusterDescriptor [] - Cancelling deployment from Deployment Failure Hook2020-11-13 18:46:52,179 INFO org.apache.hadoop.yarn.client.AHSProxy [] - Connecting to Application History server at FAT-hadoopuat-69117.vm.dc01. .tech/10.69.1.17:102002020-11-13 18:46:52,180 INFO org.apache.flink.yarn.YarnClusterDescriptor [] - Killing YARN application2020-11-13 18:46:52,184 INFO org.apache.hadoop.yarn.client.ConfiguredRMFailoverProxyProvider [] - Failing over to rm22020-11-13 18:46:52,188 INFO org.apache.hadoop.yarn.client.api.impl.YarnClientImpl [] - Killed application application_1599741232083_220232020-11-13 18:46:52,188 INFO org.apache.flink.yarn.YarnClusterDescriptor [] - Deleting files in hdfs://flashHadoopUAT/user/deploy/.flink/application_1599741232083_22023. {code} > use `yarn.provided.lib.dirs` config deploy job failed in yarn per job mode > -------------------------------------------------------------------------- > > Key: FLINK-20143 > URL: https://issues.apache.org/jira/browse/FLINK-20143 > Project: Flink > Issue Type: Bug > Components: Client / Job Submission, Deployment / YARN > Affects Versions: 1.12.0 > Reporter: zhisheng > Priority: Major > Attachments: image-2020-11-13-17-21-47-751.png, > image-2020-11-13-17-22-06-111.png, image-2020-11-13-18-43-55-188.png > > > use follow command deploy flink job to yarn failed > {code:java} > ./bin/flink run -m yarn-cluster -d -ynm flink-1.12-test -ytm 3g -yjm 3g -yD > yarn.provided.lib.dirs=hdfs:///flink/flink-1.12-SNAPSHOT/lib > ./examples/streaming/StateMachineExample.jar > {code} > log: > {code:java} > $ ./bin/flink run -m yarn-cluster -d -ynm flink-1.12-test -ytm 3g -yjm 3g -yD > yarn.provided.lib.dirs=hdfs:///flink/flink-1.12-SNAPSHOT/lib > ./examples/streaming/StateMachineExample.jar$ ./bin/flink run -m yarn-cluster > -d -ynm flink-1.12-test -ytm 3g -yjm 3g -yD > yarn.provided.lib.dirs=hdfs:///flink/flink-1.12-SNAPSHOT/lib > ./examples/streaming/StateMachineExample.jarSLF4J: Class path contains > multiple SLF4J bindings.SLF4J: Found binding in > [jar:file:/data1/app/flink-1.12-SNAPSHOT/lib/log4j-slf4j-impl-2.12.1.jar!/org/slf4j/impl/StaticLoggerBinder.class]SLF4J: > Found binding in > [jar:file:/data1/app/hadoop-2.7.3-snappy-32core12disk/share/hadoop/common/lib/slf4j-log4j12-1.7.10.jar!/org/slf4j/impl/StaticLoggerBinder.class]SLF4J: > Found binding in > [jar:file:/data1/app/hadoop-2.7.3-snappy-32core12disk/share/hadoop/tools/lib/hadoop-aliyun-2.9.2-jar-with-dependencies.jar!/org/slf4j/impl/StaticLoggerBinder.class]SLF4J: > See http://www.slf4j.org/codes.html#multiple_bindings for an > explanation.SLF4J: Actual binding is of type > [org.apache.logging.slf4j.Log4jLoggerFactory]2020-11-13 16:14:30,347 INFO > org.apache.flink.yarn.cli.FlinkYarnSessionCli [] - Dynamic > Property set: > yarn.provided.lib.dirs=hdfs:///flink/flink-1.12-SNAPSHOT/lib2020-11-13 > 16:14:30,347 INFO org.apache.flink.yarn.cli.FlinkYarnSessionCli > [] - Dynamic Property set: > yarn.provided.lib.dirs=hdfs:///flink/flink-1.12-SNAPSHOT/libUsage with > built-in data generator: StateMachineExample [--error-rate > <probability-of-invalid-transition>] [--sleep <sleep-per-record-in-ms>]Usage > with Kafka: StateMachineExample --kafka-topic <topic> [--brokers > <brokers>]Options for both the above setups: [--backend <file|rocks>] > [--checkpoint-dir <filepath>] [--async-checkpoints <true|false>] > [--incremental-checkpoints <true|false>] [--output <filepath> OR null for > stdout] > Using standalone source with error rate 0.000000 and sleep delay 1 millis > 2020-11-13 16:14:30,706 WARN > org.apache.flink.yarn.configuration.YarnLogConfigUtil [] - The > configuration directory ('/data1/app/flink-1.12-SNAPSHOT/conf') already > contains a LOG4J config file.If you want to use logback, then please delete > or rename the log configuration file.2020-11-13 16:14:30,947 INFO > org.apache.hadoop.yarn.client.AHSProxy [] - Connecting > to Application History server at > FAT-hadoopuat-69117.vm.dc01.tech/10.69.1.17:102002020-11-13 16:14:30,958 INFO > org.apache.flink.yarn.YarnClusterDescriptor [] - No path > for the flink jar passed. Using the location of class > org.apache.flink.yarn.YarnClusterDescriptor to locate the jar2020-11-13 > 16:14:31,065 INFO > org.apache.hadoop.yarn.client.ConfiguredRMFailoverProxyProvider [] - Failing > over to rm22020-11-13 16:14:31,130 INFO > org.apache.flink.yarn.YarnClusterDescriptor [] - The > configured JobManager memory is 3072 MB. YARN will allocate 4096 MB to make > up an integer multiple of its minimum allocation memory (2048 MB, configured > via 'yarn.scheduler.minimum-allocation-mb'). The extra 1024 MB may not be > used by Flink.2020-11-13 16:14:31,130 INFO > org.apache.flink.yarn.YarnClusterDescriptor [] - The > configured TaskManager memory is 3072 MB. YARN will allocate 4096 MB to make > up an integer multiple of its minimum allocation memory (2048 MB, configured > via 'yarn.scheduler.minimum-allocation-mb'). The extra 1024 MB may not be > used by Flink.2020-11-13 16:14:31,130 INFO > org.apache.flink.yarn.YarnClusterDescriptor [] - Cluster > specification: ClusterSpecification{masterMemoryMB=3072, > taskManagerMemoryMB=3072, slotsPerTaskManager=2}2020-11-13 16:14:31,681 WARN > org.apache.hadoop.hdfs.shortcircuit.DomainSocketFactory [] - The > short-circuit local reads feature cannot be used because libhadoop cannot be > loaded.2020-11-13 16:14:33,417 INFO > org.apache.flink.yarn.YarnClusterDescriptor [] - Submitting > application master application_1599741232083_219902020-11-13 16:14:33,446 > INFO org.apache.hadoop.yarn.client.api.impl.YarnClientImpl [] - > Submitted application application_1599741232083_219902020-11-13 16:14:33,446 > INFO org.apache.flink.yarn.YarnClusterDescriptor [] - > Waiting for the cluster to be allocated2020-11-13 16:14:33,448 INFO > org.apache.flink.yarn.YarnClusterDescriptor [] - Deploying > cluster, current state ACCEPTED > ------------------------------------------------------------ The program > finished with the following exception: > org.apache.flink.client.program.ProgramInvocationException: The main method > caused an error: Could not deploy Yarn job cluster. at > org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:330) > at > org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:198) > at org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:114) > at > org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:743) > at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:242) at > org.apache.flink.client.cli.CliFrontend.parseAndRun(CliFrontend.java:971) at > org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:1047) > at java.security.AccessController.doPrivileged(Native Method) at > javax.security.auth.Subject.doAs(Subject.java:422) at > org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1698) > at > org.apache.flink.runtime.security.contexts.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:41) > at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:1047)Caused > by: org.apache.flink.client.deployment.ClusterDeploymentException: Could not > deploy Yarn job cluster. at > org.apache.flink.yarn.YarnClusterDescriptor.deployJobCluster(YarnClusterDescriptor.java:460) > at > org.apache.flink.client.deployment.executors.AbstractJobClusterExecutor.execute(AbstractJobClusterExecutor.java:70) > at > org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.executeAsync(StreamExecutionEnvironment.java:1916) > at > org.apache.flink.client.program.StreamContextEnvironment.executeAsync(StreamContextEnvironment.java:128) > at > org.apache.flink.client.program.StreamContextEnvironment.execute(StreamContextEnvironment.java:76) > at > org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.java:1798) > at > org.apache.flink.streaming.examples.statemachine.StateMachineExample.main(StateMachineExample.java:142) > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) at > org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:316) > ... 11 moreCaused by: > org.apache.flink.yarn.YarnClusterDescriptor$YarnDeploymentException: The YARN > application unexpectedly switched to state FAILED during > deployment.Diagnostics from YARN: Application application_1599741232083_21990 > failed 2 times in previous 10000 milliseconds due to AM Container for > appattempt_1599741232083_21990_000002 exited with exitCode: -1Failing this > attempt.Diagnostics: [2020-11-13 16:14:38.244]Destination must be relativeFor > more detailed output, check the application tracking page: > http://FAT-hadoopuat-69117.vm.dc01.tech:8188/applicationhistory/app/application_1599741232083_21990 > Then click on links to logs of each attempt.. Failing the application.If log > aggregation is enabled on your cluster, use this command to further > investigate the issue:yarn logs -applicationId > application_1599741232083_21990 at > org.apache.flink.yarn.YarnClusterDescriptor.startAppMaster(YarnClusterDescriptor.java:1078) > at > org.apache.flink.yarn.YarnClusterDescriptor.deployInternal(YarnClusterDescriptor.java:558) > at > org.apache.flink.yarn.YarnClusterDescriptor.deployJobCluster(YarnClusterDescriptor.java:453) > ... 22 more2020-11-13 16:14:38,492 INFO > org.apache.flink.yarn.YarnClusterDescriptor [] - Cancelling > deployment from Deployment Failure Hook2020-11-13 16:14:38,494 INFO > org.apache.hadoop.yarn.client.AHSProxy [] - Connecting > to Application History server at > FAT-hadoopuat-69117.vm.dc01.tech/10.69.1.17:102002020-11-13 16:14:38,495 INFO > org.apache.flink.yarn.YarnClusterDescriptor [] - Killing > YARN application2020-11-13 16:14:38,499 INFO > org.apache.hadoop.yarn.client.ConfiguredRMFailoverProxyProvider [] - Failing > over to rm22020-11-13 16:14:38,503 INFO > org.apache.hadoop.yarn.client.api.impl.YarnClientImpl [] - Killed > application application_1599741232083_219902020-11-13 16:14:38,503 INFO > org.apache.flink.yarn.YarnClusterDescriptor [] - Deleting > files in > hdfs://flashHadoopUAT/user/deploy/.flink/application_1599741232083_21990. > {code} > but if i set `execution.target: yarn-per-job` in flink-conf.yaml, it runs ok > if i run in application mode, it runs ok too > {code:java} > ./bin/flink run-application -p 2 -d -t yarn-application -ytm 3g -yjm 3g -yD > yarn.provided.lib.dirs=hdfs:///flink/flink-1.12-SNAPSHOT/lib > ./examples/streaming/StateMachineExample.jar > {code} > but the jobid is 00000000000000000000000000000000 > > > -- This message was sent by Atlassian Jira (v8.3.4#803005)