This looks like a bug at first glance. Could you please open a ticket for that?
If not, I'd do that tomorrow. On Wed, Jun 23, 2021 at 6:36 AM Yaroslav Tkachenko < yaroslav.tkache...@shopify.com> wrote: > Hi everyone, > > I need to add support for the GCS filesystem. I have a working example > where I add two JARs to the */opt/flink/lib*/ folder: > > - GCS Hadoop connector > - *Shaded* Hadoop using flink-shaded-hadoop-2-uber-2.8.3-10.0.jar > > Now I'm trying to follow the advice from > https://ci.apache.org/projects/flink/flink-docs-release-1.13/docs/deployment/filesystems/overview/#pluggable-file-systems > and use Plugins instead. I followed the recommendation from here > https://ci.apache.org/projects/flink/flink-docs-release-1.13/docs/deployment/filesystems/plugins/. > Now I have two JARs in the* /opt/flink/plugins/hadoop-gcs/* folder: > > - GCS Hadoop connector > -* Non-shaded* Hadoop using hadoop-common-2.10.1.jar > > As I can see, shading is not required for plugins, so I want to make it > work with a simple non-shaded hadoop-common. However, JobManager fails with > the following exceptions: > > > > Caused by: org.apache.flink.runtime.client.JobInitializationException: > Could not start the JobMaster. > at > org.apache.flink.runtime.jobmaster.DefaultJobMasterServiceProcess.lambda$new$0(DefaultJobMasterServiceProcess.java:97) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at java.util.concurrent.CompletableFuture.uniWhenComplete(Unknown > Source) ~[?:?] > at > java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(Unknown > Source) ~[?:?] > at java.util.concurrent.CompletableFuture.postComplete(Unknown Source) > ~[?:?] > at java.util.concurrent.CompletableFuture$AsyncSupply.run(Unknown > Source) ~[?:?] > at java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source) > ~[?:?] > at java.util.concurrent.FutureTask.run(Unknown Source) ~[?:?] > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(Unknown > Source) ~[?:?] > at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source) > ~[?:?] > at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source) > ~[?:?] > at java.lang.Thread.run(Unknown Source) ~[?:?] > Caused by: java.util.concurrent.CompletionException: > org.apache.flink.util.FlinkRuntimeException: Failed to create checkpoint > storage at checkpoint coordinator side. > at java.util.concurrent.CompletableFuture.encodeThrowable(Unknown > Source) ~[?:?] > at java.util.concurrent.CompletableFuture.completeThrowable(Unknown > Source) ~[?:?] > at java.util.concurrent.CompletableFuture$AsyncSupply.run(Unknown > Source) ~[?:?] > at java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source) > ~[?:?] > at java.util.concurrent.FutureTask.run(Unknown Source) ~[?:?] > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(Unknown > Source) ~[?:?] > at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source) > ~[?:?] > at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source) > ~[?:?] > at java.lang.Thread.run(Unknown Source) ~[?:?] > Caused by: org.apache.flink.util.FlinkRuntimeException: Failed to create > checkpoint storage at checkpoint coordinator side. > at > org.apache.flink.runtime.checkpoint.CheckpointCoordinator.<init>(CheckpointCoordinator.java:324) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.checkpoint.CheckpointCoordinator.<init>(CheckpointCoordinator.java:240) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.executiongraph.DefaultExecutionGraph.enableCheckpointing(DefaultExecutionGraph.java:448) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.executiongraph.DefaultExecutionGraphBuilder.buildGraph(DefaultExecutionGraphBuilder.java:311) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.scheduler.DefaultExecutionGraphFactory.createAndRestoreExecutionGraph(DefaultExecutionGraphFactory.java:107) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.scheduler.SchedulerBase.createAndRestoreExecutionGraph(SchedulerBase.java:342) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.scheduler.SchedulerBase.<init>(SchedulerBase.java:190) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.scheduler.DefaultScheduler.<init>(DefaultScheduler.java:120) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.scheduler.DefaultSchedulerFactory.createInstance(DefaultSchedulerFactory.java:132) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.jobmaster.DefaultSlotPoolServiceSchedulerFactory.createScheduler(DefaultSlotPoolServiceSchedulerFactory.java:110) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.jobmaster.JobMaster.createScheduler(JobMaster.java:340) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.jobmaster.JobMaster.<init>(JobMaster.java:317) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.jobmaster.factories.DefaultJobMasterServiceFactory.internalCreateJobMasterService(DefaultJobMasterServiceFactory.java:107) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.jobmaster.factories.DefaultJobMasterServiceFactory.lambda$createJobMasterService$0(DefaultJobMasterServiceFactory.java:95) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.util.function.FunctionUtils.lambda$uncheckedSupplier$4(FunctionUtils.java:112) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at java.util.concurrent.CompletableFuture$AsyncSupply.run(Unknown > Source) ~[?:?] > at java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source) > ~[?:?] > at java.util.concurrent.FutureTask.run(Unknown Source) ~[?:?] > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(Unknown > Source) ~[?:?] > at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source) > ~[?:?] > at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source) > ~[?:?] > at java.lang.Thread.run(Unknown Source) ~[?:?] > Caused by: org.apache.flink.core.fs.UnsupportedFileSystemSchemeException: > Could not find a file system implementation for scheme 'gs'. 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:530) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at org.apache.flink.core.fs.FileSystem.get(FileSystem.java:407) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at org.apache.flink.core.fs.Path.getFileSystem(Path.java:274) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.state.filesystem.FsCheckpointStorageAccess.<init>(FsCheckpointStorageAccess.java:64) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.state.storage.FileSystemCheckpointStorage.createCheckpointStorage(FileSystemCheckpointStorage.java:323) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.checkpoint.CheckpointCoordinator.<init>(CheckpointCoordinator.java:321) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.checkpoint.CheckpointCoordinator.<init>(CheckpointCoordinator.java:240) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.executiongraph.DefaultExecutionGraph.enableCheckpointing(DefaultExecutionGraph.java:448) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.executiongraph.DefaultExecutionGraphBuilder.buildGraph(DefaultExecutionGraphBuilder.java:311) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.scheduler.DefaultExecutionGraphFactory.createAndRestoreExecutionGraph(DefaultExecutionGraphFactory.java:107) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.scheduler.SchedulerBase.createAndRestoreExecutionGraph(SchedulerBase.java:342) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.scheduler.SchedulerBase.<init>(SchedulerBase.java:190) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.scheduler.DefaultScheduler.<init>(DefaultScheduler.java:120) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.scheduler.DefaultSchedulerFactory.createInstance(DefaultSchedulerFactory.java:132) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.jobmaster.DefaultSlotPoolServiceSchedulerFactory.createScheduler(DefaultSlotPoolServiceSchedulerFactory.java:110) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.jobmaster.JobMaster.createScheduler(JobMaster.java:340) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.jobmaster.JobMaster.<init>(JobMaster.java:317) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.jobmaster.factories.DefaultJobMasterServiceFactory.internalCreateJobMasterService(DefaultJobMasterServiceFactory.java:107) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.jobmaster.factories.DefaultJobMasterServiceFactory.lambda$createJobMasterService$0(DefaultJobMasterServiceFactory.java:95) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.util.function.FunctionUtils.lambda$uncheckedSupplier$4(FunctionUtils.java:112) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at java.util.concurrent.CompletableFuture$AsyncSupply.run(Unknown > Source) ~[?:?] > at java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source) > ~[?:?] > at java.util.concurrent.FutureTask.run(Unknown Source) ~[?:?] > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(Unknown > Source) ~[?:?] > at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source) > ~[?:?] > at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source) > ~[?:?] > at java.lang.Thread.run(Unknown Source) ~[?:?] > 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:55) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.core.fs.FileSystem.getUnguardedFileSystem(FileSystem.java:526) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at org.apache.flink.core.fs.FileSystem.get(FileSystem.java:407) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at org.apache.flink.core.fs.Path.getFileSystem(Path.java:274) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.state.filesystem.FsCheckpointStorageAccess.<init>(FsCheckpointStorageAccess.java:64) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.state.storage.FileSystemCheckpointStorage.createCheckpointStorage(FileSystemCheckpointStorage.java:323) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.checkpoint.CheckpointCoordinator.<init>(CheckpointCoordinator.java:321) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.checkpoint.CheckpointCoordinator.<init>(CheckpointCoordinator.java:240) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.executiongraph.DefaultExecutionGraph.enableCheckpointing(DefaultExecutionGraph.java:448) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.executiongraph.DefaultExecutionGraphBuilder.buildGraph(DefaultExecutionGraphBuilder.java:311) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.scheduler.DefaultExecutionGraphFactory.createAndRestoreExecutionGraph(DefaultExecutionGraphFactory.java:107) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.scheduler.SchedulerBase.createAndRestoreExecutionGraph(SchedulerBase.java:342) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.scheduler.SchedulerBase.<init>(SchedulerBase.java:190) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.scheduler.DefaultScheduler.<init>(DefaultScheduler.java:120) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.scheduler.DefaultSchedulerFactory.createInstance(DefaultSchedulerFactory.java:132) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.jobmaster.DefaultSlotPoolServiceSchedulerFactory.createScheduler(DefaultSlotPoolServiceSchedulerFactory.java:110) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.jobmaster.JobMaster.createScheduler(JobMaster.java:340) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.jobmaster.JobMaster.<init>(JobMaster.java:317) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.jobmaster.factories.DefaultJobMasterServiceFactory.internalCreateJobMasterService(DefaultJobMasterServiceFactory.java:107) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.runtime.jobmaster.factories.DefaultJobMasterServiceFactory.lambda$createJobMasterService$0(DefaultJobMasterServiceFactory.java:95) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at > org.apache.flink.util.function.FunctionUtils.lambda$uncheckedSupplier$4(FunctionUtils.java:112) > ~[flink-dist_2.12-1.13.0.jar:1.13.0] > at java.util.concurrent.CompletableFuture$AsyncSupply.run(Unknown > Source) ~[?:?] > at java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source) > ~[?:?] > at java.util.concurrent.FutureTask.run(Unknown Source) ~[?:?] > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(Unknown > Source) ~[?:?] > at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source) > ~[?:?] > at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source) > ~[?:?] > at java.lang.Thread.run(Unknown Source) ~[?:?] > > > > The exception is thrown when org.apache.hadoop.conf.Configuration > and org.apache.hadoop.fs.FileSystem are not available in the classpath ( > https://github.com/apache/flink/blob/f2f2befee76d08b4d9aa592438dc0cf5ebe2ef96/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java#L1123-L1124), > but they're available in hadoop-common and should have been loaded. > > What should I do to make it work? > > Thanks! > >