-
Notifications
You must be signed in to change notification settings - Fork 468
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[VL] The POC of supporting Flink in Gluten #8839
base: main
Are you sure you want to change the base?
Changes from 5 commits
396abba
e516d49
3afead5
974b972
4a15f3f
a6e9540
f6721f1
776f253
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,97 @@ | ||
--- | ||
layout: page | ||
title: Gluten For Flink with Velox Backend | ||
nav_order: 1 | ||
parent: Getting-Started | ||
--- | ||
|
||
# Supported Version | ||
|
||
| Type | Version | | ||
|-------|------------------------------| | ||
| Flink | 1.20 | | ||
| OS | Ubuntu20.04/22.04, Centos7/8 | | ||
| jdk | openjdk11/jdk17 | | ||
| scala | 2.12 | | ||
|
||
# Prerequisite | ||
|
||
Currently, with static build Gluten+Flink+Velox backend supports all the Linux OSes, but is only tested on **Ubuntu20.04/Ubuntu22.04/Centos7/Centos8**. With dynamic build, Gluten+Velox backend support **Ubuntu20.04/Ubuntu22.04/Centos7/Centos8** and their variants. | ||
|
||
Currently, the officially supported Flink versions are 1.20.*. | ||
|
||
We need to set up the `JAVA_HOME` env. Currently, Gluten supports **java 11** and **java 17**. | ||
|
||
**For x86_64** | ||
|
||
```bash | ||
## make sure jdk8 is used | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The comment is not aligned with the following. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. done |
||
export JAVA_HOME=/usr/lib/jvm/java-11-openjdk-amd64 | ||
export PATH=$JAVA_HOME/bin:$PATH | ||
``` | ||
|
||
**For aarch64** | ||
|
||
```bash | ||
## make sure jdk8 is used | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ditto. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. done |
||
export JAVA_HOME=/usr/lib/jvm/java-11-openjdk-arm64 | ||
export PATH=$JAVA_HOME/bin:$PATH | ||
``` | ||
|
||
**Get gluten** | ||
|
||
```bash | ||
## config maven, like proxy in ~/.m2/settings.xml | ||
|
||
## fetch gluten code | ||
git clone https://github.com/apache/incubator-gluten.git | ||
``` | ||
|
||
# Build Gluten Flink with Velox Backend | ||
|
||
``` | ||
cd /path/to/gluten/gluten-flink | ||
mvn clean package | ||
``` | ||
|
||
## Dependency library deployment | ||
|
||
Gluten for Flink depends on [Velox4j](https://github.com/velox4j/velox4j) to call velox. So you need to get the Velox4j packages and used them with gluten. | ||
Velox4j jar available now is velox4j-0.1.0-SNAPSHOT.jar. | ||
|
||
## Submit the Flink SQL job | ||
|
||
Submit test script from `flink run`. You can use the `StreamSQLExample` as an example. | ||
|
||
### Flink local cluster | ||
``` | ||
var parquet_file_path = "/PATH/TO/TPCH_PARQUET_PATH" | ||
var gluten_root = "/PATH/TO/GLUTEN" | ||
``` | ||
|
||
After deploying flink binaries, please add gluten-flink jar to flink library path, | ||
including gluten-flink-runtime-1.4.0.jar, gluten-flink-loader-1.4.0.jar and Velox4j jars above. | ||
And make them loaded before flink libraries. | ||
Then you can go to flink binary path and use the below scripts to | ||
submit the example job. | ||
|
||
```bash | ||
bin/start-cluster.sh | ||
bin/flink run -d -m 0.0.0.0:8080 \ | ||
-c org.apache.flink.table.examples.java.basics.StreamSQLExample \ | ||
lib/flink-examples-table_2.12-1.20.1.jar | ||
``` | ||
|
||
Then you can get the result in `log/flink-*-taskexecutor-*.out`. | ||
And you can see an operator named `gluten-cal` from the web frontend of your flink job. | ||
|
||
### Flink Yarn per job mode | ||
|
||
TODO | ||
|
||
## Notes: | ||
Now both Gluten for Flink and Velox4j have not a bundled jar including all jar depends on. | ||
So you may have to add these jars by yourself, which may including guava-33.4.0-jre.jar, jackson-core-2.18.0.jar, | ||
jackson-databind-2.18.0.jar, jackson-datatype-jdk8-2.18.0.jar, jackson-annotations-2.18.0.jar, arrow-memory-core-18.1.0.jar, | ||
arrow-memory-unsafe-18.1.0.jar, arrow-vector-18.1.0.jar, flatbuffers-java-24.3.25.jar, arrow-format-18.1.0.jar, arrow-c-data-18.1.0.jar. | ||
We will supply bundled jars soon. |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,2 @@ | ||
# Gluten Flink Project | ||
Gluten for Flink is under developing now, you can refer to [user guide](../docs/get-started/Flink.md) for a quick usage. |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,82 @@ | ||
<?xml version="1.0" encoding="UTF-8"?> | ||
<!-- | ||
Licensed under the Apache License, Version 2.0 (the "License"); | ||
you may not use this file except in compliance with the License. | ||
You may obtain a copy of the License at | ||
|
||
http://www.apache.org/licenses/LICENSE-2.0 | ||
|
||
Unless required by applicable law or agreed to in writing, software | ||
distributed under the License is distributed on an "AS IS" BASIS, | ||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
See the License for the specific language governing permissions and | ||
limitations under the License. | ||
--> | ||
<project xmlns="http://maven.apache.org/POM/4.0.0" | ||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" | ||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> | ||
<modelVersion>4.0.0</modelVersion> | ||
|
||
<parent> | ||
<groupId>org.apache.gluten</groupId> | ||
<artifactId>gluten-flink</artifactId> | ||
<version>1.4.0-SNAPSHOT</version> | ||
<relativePath>../pom.xml</relativePath> | ||
</parent> | ||
|
||
<artifactId>gluten-flink-loader</artifactId> | ||
<name>Gluten Flink Loader</name> | ||
<packaging>jar</packaging> | ||
|
||
<dependencies> | ||
<dependency> | ||
<groupId>org.apache.flink</groupId> | ||
<artifactId>flink-core</artifactId> | ||
<version>${flink.version}</version> | ||
<scope>provided</scope> | ||
</dependency> | ||
<dependency> | ||
<groupId>org.apache.flink</groupId> | ||
<artifactId>flink-table-planner_${scala.binary.version}</artifactId> | ||
<version>${flink.version}</version> | ||
<scope>provided</scope> | ||
</dependency> | ||
<dependency> | ||
<groupId>org.apache.gluten</groupId> | ||
<artifactId>gluten-flink-planner</artifactId> | ||
<version>${project.version}</version> | ||
<scope>runtime</scope> | ||
</dependency> | ||
</dependencies> | ||
|
||
<build> | ||
<plugins> | ||
<plugin> | ||
<groupId>org.apache.maven.plugins</groupId> | ||
<artifactId>maven-dependency-plugin</artifactId> | ||
<executions> | ||
<execution> | ||
<id>copy-table-planner-jars</id> | ||
<phase>prepare-package</phase> | ||
<goals> | ||
<goal>copy</goal> | ||
</goals> | ||
<configuration> | ||
<artifactItems> | ||
<artifactItem> | ||
<groupId>org.apache.gluten</groupId> | ||
<artifactId>gluten-flink-planner</artifactId> | ||
<version>${project.version}</version> | ||
<type>jar</type> | ||
<overWrite>true</overWrite> | ||
<destFileName>gluten-flink-planner.jar</destFileName> | ||
</artifactItem> | ||
</artifactItems> | ||
<outputDirectory>${project.build.directory}/classes</outputDirectory> | ||
</configuration> | ||
</execution> | ||
</executions> | ||
</plugin> | ||
</plugins> | ||
</build> | ||
</project> |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,200 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.flink.table.planner.loader; | ||
|
||
import org.apache.flink.configuration.Configuration; | ||
import org.apache.flink.configuration.ConfigurationUtils; | ||
import org.apache.flink.configuration.CoreOptions; | ||
import org.apache.flink.core.classloading.ComponentClassLoader; | ||
import org.apache.flink.table.api.TableException; | ||
import org.apache.flink.table.delegation.ExecutorFactory; | ||
import org.apache.flink.table.delegation.PlannerFactory; | ||
import org.apache.flink.table.factories.FactoryUtil; | ||
import org.apache.flink.util.FileUtils; | ||
import org.apache.flink.util.IOUtils; | ||
|
||
import java.io.IOException; | ||
import java.io.InputStream; | ||
import java.net.URL; | ||
import java.nio.file.Files; | ||
import java.nio.file.Path; | ||
import java.nio.file.Paths; | ||
import java.util.Arrays; | ||
import java.util.HashMap; | ||
import java.util.Map; | ||
import java.util.UUID; | ||
import java.util.stream.Stream; | ||
|
||
/** | ||
* This class will overwrite the PlannerModule in Flink to load gluten-flink-planner.jar. | ||
* So that it will load the classes in gluten code first. | ||
*/ | ||
class PlannerModule { | ||
|
||
/** | ||
* The name of the table planner dependency jar, bundled with flink-table-planner-loader module | ||
* artifact. | ||
*/ | ||
static final String FLINK_TABLE_PLANNER_FAT_JAR = "flink-table-planner.jar"; | ||
|
||
private static final String HINT_USAGE = | ||
"mvn clean package -pl flink-table/flink-table-planner,flink-table/flink-table-planner-loader -DskipTests"; | ||
|
||
private static final String[] OWNER_CLASSPATH = | ||
Stream.concat( | ||
Arrays.stream(CoreOptions.PARENT_FIRST_LOGGING_PATTERNS), | ||
Stream.of( | ||
// These packages are shipped either by | ||
// flink-table-runtime or flink-dist itself | ||
"org.codehaus.janino", | ||
"org.codehaus.commons", | ||
"org.apache.commons.lang3", | ||
"org.apache.commons.math3", | ||
// with hive dialect, hadoop jar should be in classpath, | ||
// also, we should make it loaded by owner classloader, | ||
// otherwise, it'll throw class not found exception | ||
// when initialize HiveParser which requires hadoop | ||
"org.apache.hadoop")) | ||
.toArray(String[]::new); | ||
|
||
private static final String[] COMPONENT_CLASSPATH = new String[] { | ||
"org.apache.flink", | ||
"org.apache.gluten", | ||
"io.github", | ||
"com.google" | ||
}; | ||
|
||
private static final Map<String, String> KNOWN_MODULE_ASSOCIATIONS = new HashMap<>(); | ||
|
||
static { | ||
KNOWN_MODULE_ASSOCIATIONS.put("org.apache.gluten.table.runtime", "gluten-flink-runtime"); | ||
KNOWN_MODULE_ASSOCIATIONS.put("org.apache.flink.table.runtime", "flink-table-runtime"); | ||
KNOWN_MODULE_ASSOCIATIONS.put("org.apache.flink.formats.raw", "flink-table-runtime"); | ||
|
||
KNOWN_MODULE_ASSOCIATIONS.put("org.codehaus.janino", "flink-table-runtime"); | ||
KNOWN_MODULE_ASSOCIATIONS.put("org.codehaus.commons", "flink-table-runtime"); | ||
KNOWN_MODULE_ASSOCIATIONS.put( | ||
"org.apache.flink.table.shaded.com.jayway", "flink-table-runtime"); | ||
} | ||
|
||
private final PlannerComponentClassLoader submoduleClassLoader; | ||
|
||
private PlannerModule() { | ||
try { | ||
final ClassLoader flinkClassLoader = PlannerModule.class.getClassLoader(); | ||
|
||
final Path tmpDirectory = | ||
Paths.get(ConfigurationUtils.parseTempDirectories(new Configuration())[0]); | ||
Files.createDirectories(FileUtils.getTargetPathIfContainsSymbolicPath(tmpDirectory)); | ||
final Path tempFile = | ||
Files.createFile( | ||
tmpDirectory.resolve( | ||
"flink-table-planner_" + UUID.randomUUID() + ".jar")); | ||
|
||
final InputStream resourceStream = | ||
flinkClassLoader.getResourceAsStream(FLINK_TABLE_PLANNER_FAT_JAR); | ||
InputStream glutenStream = | ||
flinkClassLoader.getResourceAsStream("gluten-flink-planner.jar"); | ||
if (resourceStream == null || glutenStream == null) { | ||
throw new TableException( | ||
String.format( | ||
"Flink Table planner could not be found. If this happened while running a test in the IDE, " | ||
+ "run '%s' on the command-line, " | ||
+ "or add a test dependency on the flink-table-planner-loader test-jar.", | ||
HINT_USAGE)); | ||
} | ||
final Path glutenFile = | ||
Files.createFile( | ||
tmpDirectory.resolve( | ||
"gluten-flink-planner_" + UUID.randomUUID() + ".jar")); | ||
|
||
IOUtils.copyBytes(resourceStream, Files.newOutputStream(tempFile)); | ||
IOUtils.copyBytes(glutenStream, Files.newOutputStream(glutenFile)); | ||
tempFile.toFile().deleteOnExit(); | ||
glutenFile.toFile().deleteOnExit(); | ||
|
||
this.submoduleClassLoader = | ||
new PlannerComponentClassLoader( | ||
new URL[] { | ||
glutenFile.toUri().toURL(), | ||
tempFile.toUri().toURL()}, | ||
flinkClassLoader, | ||
OWNER_CLASSPATH, | ||
COMPONENT_CLASSPATH, | ||
KNOWN_MODULE_ASSOCIATIONS); | ||
} catch (IOException e) { | ||
throw new TableException( | ||
"Could not initialize the table planner components loader.", e); | ||
} | ||
} | ||
|
||
public void addUrlToClassLoader(URL url) { | ||
// add the url to component url | ||
this.submoduleClassLoader.addURL(url); | ||
} | ||
|
||
// Singleton lazy initialization | ||
|
||
private static class PlannerComponentsHolder { | ||
private static final PlannerModule INSTANCE = new PlannerModule(); | ||
} | ||
|
||
public static PlannerModule getInstance() { | ||
return PlannerComponentsHolder.INSTANCE; | ||
} | ||
|
||
// load methods for various components provided by the planner | ||
|
||
public ExecutorFactory loadExecutorFactory() { | ||
return FactoryUtil.discoverFactory( | ||
this.submoduleClassLoader, | ||
ExecutorFactory.class, | ||
ExecutorFactory.DEFAULT_IDENTIFIER); | ||
} | ||
|
||
public PlannerFactory loadPlannerFactory() { | ||
return FactoryUtil.discoverFactory( | ||
this.submoduleClassLoader, PlannerFactory.class, PlannerFactory.DEFAULT_IDENTIFIER); | ||
} | ||
|
||
/** | ||
* A class loader extending {@link ComponentClassLoader} which overwrites method{@link #addURL} | ||
* to enable it can add url to component classloader. | ||
*/ | ||
private static class PlannerComponentClassLoader extends ComponentClassLoader { | ||
|
||
public PlannerComponentClassLoader( | ||
URL[] classpath, | ||
ClassLoader ownerClassLoader, | ||
String[] ownerFirstPackages, | ||
String[] componentFirstPackages, | ||
Map<String, String> knownPackagePrefixesModuleAssociation) { | ||
super( | ||
classpath, | ||
ownerClassLoader, | ||
ownerFirstPackages, | ||
componentFirstPackages, | ||
knownPackagePrefixesModuleAssociation); | ||
} | ||
|
||
@Override | ||
public void addURL(URL url) { | ||
super.addURL(url); | ||
} | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I prefer firstly putting this doc under gluten-flink/docs folder as it is an experimental and separate feature.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
OK