Kengo Seki created BIGTOP-3571:
----------------------------------

             Summary: Ensure Hadoop and Flink are deployed in the correct order
                 Key: BIGTOP-3571
                 URL: https://issues.apache.org/jira/browse/BIGTOP-3571
             Project: Bigtop
          Issue Type: Bug
          Components: deployment, flink, hadoop
            Reporter: Kengo Seki
            Assignee: Kengo Seki


Flink's smoke test frequently fails on our CI though it seems to be deployed 
without errors, e.g.

https://ci.bigtop.apache.org/view/Test/job/Bigtop-trunk-smoke-tests/245/COMPONENTS=hdfs.flink@flink,OS=centos-8-x86_64-deploy/console
https://ci.bigtop.apache.org/view/Test/job/Bigtop-trunk-smoke-tests/245/COMPONENTS=hdfs.flink@flink,OS=debian-9-amd64-deploy/console
https://ci.bigtop.apache.org/view/Test/job/Bigtop-trunk-smoke-tests/245/COMPONENTS=hdfs.flink@flink,OS=fedora-33-x86_64-deploy/console
https://ci.bigtop.apache.org/view/Test/job/Bigtop-trunk-smoke-tests/245/COMPONENTS=hdfs.flink@flink,OS=ubuntu-18.04-amd64-deploy/console

I logged into the container in question and found the following error:

{code}
root@81ad5dcbab25:/# cat 
/var/log/flink/flink-flink-standalonesession-0-81ad5dcbab25.log 

...

2021-07-11 00:57:03,082 ERROR 
org.apache.flink.runtime.dispatcher.StandaloneDispatcher     [] - Failed to 
submit job c23898841852bceb0b64968470ff906e.
org.apache.flink.runtime.client.JobExecutionException: Could not instantiate 
JobManager.
        at 
org.apache.flink.runtime.dispatcher.Dispatcher.lambda$createJobManagerRunner$6(Dispatcher.java:427)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1604)
 ~[?:1.8.0_292]
        at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40) 
[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(ForkJoinExecutorConfigurator.scala:44)
 [flink-dist_2.11-1.11.3.jar:1.11.3]
        at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) 
[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339) 
[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979) 
[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107) 
[flink-dist_2.11-1.11.3.jar:1.11.3]
Caused by: org.apache.flink.runtime.client.JobExecutionException: Cannot 
initialize task 'DataSink (CsvOutputFormat (path: 
hdfs://81ad5dcbab25.bigtop.apache.org:8020/tmp/result.txt, delimiter:  ))': 
Could not find a file system implementation for scheme 'hdfs'. The scheme is 
not directly supported by Flink and no Hadoop file system to support this 
scheme could be loaded. For a full list of supported file systems, please see 
https://ci.apache.org/projects/flink/flink-docs-stable/ops/filesystems/.
        at 
org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder.buildGraph(ExecutionGraphBuilder.java:216)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.scheduler.SchedulerBase.createExecutionGraph(SchedulerBase.java:270)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.scheduler.SchedulerBase.createAndRestoreExecutionGraph(SchedulerBase.java:244)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.scheduler.SchedulerBase.<init>(SchedulerBase.java:231) 
~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.scheduler.DefaultScheduler.<init>(DefaultScheduler.java:119)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.scheduler.DefaultSchedulerFactory.createInstance(DefaultSchedulerFactory.java:103)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.jobmaster.JobMaster.createScheduler(JobMaster.java:290)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.jobmaster.JobMaster.<init>(JobMaster.java:278) 
~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.jobmaster.factories.DefaultJobMasterServiceFactory.createJobMasterService(DefaultJobMasterServiceFactory.java:98)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.jobmaster.factories.DefaultJobMasterServiceFactory.createJobMasterService(DefaultJobMasterServiceFactory.java:40)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.jobmaster.JobManagerRunnerImpl.<init>(JobManagerRunnerImpl.java:140)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.dispatcher.DefaultJobManagerRunnerFactory.createJobManagerRunner(DefaultJobManagerRunnerFactory.java:84)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.dispatcher.Dispatcher.lambda$createJobManagerRunner$6(Dispatcher.java:417)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        ... 7 more
Caused by: org.apache.flink.core.fs.UnsupportedFileSystemSchemeException: Could 
not find a file system implementation for scheme 'hdfs'. The scheme is not 
directly supported by Flink and no Hadoop file system to support this scheme 
could be loaded. For a full list of supported file systems, please see 
https://ci.apache.org/projects/flink/flink-docs-stable/ops/filesystems/.
        at 
org.apache.flink.core.fs.FileSystem.getUnguardedFileSystem(FileSystem.java:491) 
~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at org.apache.flink.core.fs.FileSystem.get(FileSystem.java:389) 
~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at org.apache.flink.core.fs.Path.getFileSystem(Path.java:292) 
~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.api.common.io.FileOutputFormat.initializeGlobal(FileOutputFormat.java:275)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.jobgraph.InputOutputFormatVertex.initializeOnMaster(InputOutputFormatVertex.java:99)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder.buildGraph(ExecutionGraphBuilder.java:212)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.scheduler.SchedulerBase.createExecutionGraph(SchedulerBase.java:270)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.scheduler.SchedulerBase.createAndRestoreExecutionGraph(SchedulerBase.java:244)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.scheduler.SchedulerBase.<init>(SchedulerBase.java:231) 
~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.scheduler.DefaultScheduler.<init>(DefaultScheduler.java:119)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.scheduler.DefaultSchedulerFactory.createInstance(DefaultSchedulerFactory.java:103)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.jobmaster.JobMaster.createScheduler(JobMaster.java:290)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.jobmaster.JobMaster.<init>(JobMaster.java:278) 
~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.jobmaster.factories.DefaultJobMasterServiceFactory.createJobMasterService(DefaultJobMasterServiceFactory.java:98)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.jobmaster.factories.DefaultJobMasterServiceFactory.createJobMasterService(DefaultJobMasterServiceFactory.java:40)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.jobmaster.JobManagerRunnerImpl.<init>(JobManagerRunnerImpl.java:140)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.dispatcher.DefaultJobManagerRunnerFactory.createJobManagerRunner(DefaultJobManagerRunnerFactory.java:84)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.dispatcher.Dispatcher.lambda$createJobManagerRunner$6(Dispatcher.java:417)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        ... 7 more
Caused by: org.apache.flink.core.fs.UnsupportedFileSystemSchemeException: 
Hadoop is not in the classpath/dependencies.
        at 
org.apache.flink.core.fs.UnsupportedSchemeFactory.create(UnsupportedSchemeFactory.java:58)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.core.fs.FileSystem.getUnguardedFileSystem(FileSystem.java:487) 
~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at org.apache.flink.core.fs.FileSystem.get(FileSystem.java:389) 
~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at org.apache.flink.core.fs.Path.getFileSystem(Path.java:292) 
~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.api.common.io.FileOutputFormat.initializeGlobal(FileOutputFormat.java:275)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.jobgraph.InputOutputFormatVertex.initializeOnMaster(InputOutputFormatVertex.java:99)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder.buildGraph(ExecutionGraphBuilder.java:212)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.scheduler.SchedulerBase.createExecutionGraph(SchedulerBase.java:270)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.scheduler.SchedulerBase.createAndRestoreExecutionGraph(SchedulerBase.java:244)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.scheduler.SchedulerBase.<init>(SchedulerBase.java:231) 
~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.scheduler.DefaultScheduler.<init>(DefaultScheduler.java:119)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.scheduler.DefaultSchedulerFactory.createInstance(DefaultSchedulerFactory.java:103)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.jobmaster.JobMaster.createScheduler(JobMaster.java:290)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.jobmaster.JobMaster.<init>(JobMaster.java:278) 
~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.jobmaster.factories.DefaultJobMasterServiceFactory.createJobMasterService(DefaultJobMasterServiceFactory.java:98)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.jobmaster.factories.DefaultJobMasterServiceFactory.createJobMasterService(DefaultJobMasterServiceFactory.java:40)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.jobmaster.JobManagerRunnerImpl.<init>(JobManagerRunnerImpl.java:140)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.dispatcher.DefaultJobManagerRunnerFactory.createJobManagerRunner(DefaultJobManagerRunnerFactory.java:84)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        at 
org.apache.flink.runtime.dispatcher.Dispatcher.lambda$createJobManagerRunner$6(Dispatcher.java:417)
 ~[flink-dist_2.11-1.11.3.jar:1.11.3]
        ... 7 more
{code}

[Flink is integrated with Hadoop through the HADOOP_CLASSPATH 
envvar|https://ci.apache.org/projects/flink/flink-docs-release-1.11/ops/deployment/hadoop.html#providing-hadoop-classes].
 But [if Flink services start up too early than the hadoop installation, that 
envvar is not 
set|https://github.com/apache/bigtop/blob/master/bigtop-packages/src/common/flink/flink-jobmanager.svc#L34-L36]
 and it causes the error above.
So we have to set them up in the correct order if they are deployed 
simultaneously.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to