Plugins

Plugins facilitate a strict separation of code through restricted classloaders. Plugins cannotaccess classes from other plugins or from Flink that have not been specifically whitelisted. Thisstrict isolation allows plugins to contain conflicting versions of the same library without the needto relocate classes or to converge to common versions. Currently, only file systems are pluggablebut in the future, connectors, formats, and even user code should also be pluggable.

Isolation and plugin structure

Plugins reside in their own folders and can consist of several jars. The names of the plugin foldersare arbitrary.

  1. flink-dist
  2. ├── conf
  3. ├── lib
  4. ...
  5. └── plugins
  6. ├── s3
  7. ├── aws-credential-provider.jar
  8. └── flink-s3-fs-hadoop.jar
  9. └── azure
  10. └── flink-azure-fs-hadoop.jar

Each plugin is loaded through its own classloader and completely isolated from any other plugin.Hence, the flink-s3-fs-hadoop and flink-azure-fs-hadoop can depend on different conflictinglibrary versions. There is no need to relocate any class during the creation of fat jars (shading).

Plugins may access certain whitelisted packages from Flink’s lib/ folder. In particular, allnecessary service provider interfaces (SPI) are loaded through the system classloader, so that notwo versions of org.apache.flink.core.fs.FileSystem exist at any given time, even if usersaccidentally bundle it in their fat jar. This singleton class requirement is strictly necessary sothat the Flink runtime has an entry point into the plugin. Service classes are discovered throughthe java.util.ServiceLoader, so make sure to retain the service definitions in META-INF/servicesduring shading.

NoteCurrently, more Flink core classes are stillaccessible from plugins as we flesh out the SPI system.

Furthermore, the most common logger frameworks are whitelisted, such that logging is uniformlypossible across Flink core, plugins, and user code.

File Systems

All file systemsexcept MapR are pluggable. That means they can and shouldbe used as plugins. To use a pluggable file system, copy the corresponding JAR file from the optdirectory to a directory under plugins directory of your Flink distribution before starting Flink,e.g.

  1. mkdir ./plugins/s3-fs-hadoop
  2. cp ./opt/flink-s3-fs-hadoop-1.10.0.jar ./plugins/s3-fs-hadoop/

WarningThe s3 file systems (flink-s3-fs-presto andflink-s3-fs-hadoop) can only be used as plugins as we already removed the relocations. Placingthem in libs/ will result in system failures.

AttentionBecause of the strict isolation, file systems do not have access to credential providers in lib/anymore. Please add any needed providers to the respective plugin folder.