This is an automated email from the ASF dual-hosted git repository.
jark pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/fluss.git
The following commit(s) were added to refs/heads/main by this push:
new 78ba99b70 [filesystem] Support use Hadoop dependencies from
environment variables HADOOP_CLASSPATH (#1359)
78ba99b70 is described below
commit 78ba99b70cbb73519859e60ce255c4166c076ec4
Author: Junbo Wang <[email protected]>
AuthorDate: Sun Sep 28 20:57:00 2025 +0800
[filesystem] Support use Hadoop dependencies from environment variables
HADOOP_CLASSPATH (#1359)
---
.../main/java/org/apache/fluss/config/ConfigOptions.java | 8 +++++++-
fluss-dist/src/main/resources/bin/config.sh | 5 +++++
fluss-lake/fluss-lake-iceberg/pom.xml | 6 ++++++
fluss-lake/fluss-lake-paimon/pom.xml | 6 ++++++
fluss-server/pom.xml | 6 ++++++
website/docs/maintenance/filesystems/hdfs.md | 14 ++++++++++++++
.../docs/maintenance/tiered-storage/lakehouse-storage.md | 7 ++++++-
7 files changed, 50 insertions(+), 2 deletions(-)
diff --git
a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java
b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java
index 8b4664d70..9f812d5d4 100644
--- a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java
+++ b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java
@@ -105,8 +105,14 @@ public class ConfigOptions {
.asList()
.defaultValues(
ArrayUtils.concat(
+ // TODO: remove core-site after implement
fluss hdfs security
+ // utils
new String[] {
- "java.", "org.apache.fluss.",
"javax.annotation."
+ "java.",
+ "org.apache.fluss.",
+ "javax.annotation.",
+ "org.apache.hadoop.",
+ "core-site",
},
PARENT_FIRST_LOGGING_PATTERNS))
.withDescription(
diff --git a/fluss-dist/src/main/resources/bin/config.sh
b/fluss-dist/src/main/resources/bin/config.sh
index 56e1e9948..1c309b52a 100755
--- a/fluss-dist/src/main/resources/bin/config.sh
+++ b/fluss-dist/src/main/resources/bin/config.sh
@@ -32,6 +32,11 @@ constructFlussClassPath() {
fi
done < <(find "$FLUSS_LIB_DIR" ! -type d -name '*.jar' -print0 | sort -z)
+ # Add Hadoop dependencies from environment variables HADOOP_CLASSPATH
+ if [ -n "${HADOOP_CLASSPATH}" ]; then
+ FLUSS_CLASSPATH="$FLUSS_CLASSPATH":"$HADOOP_CLASSPATH"
+ fi
+
local FLUSS_SERVER_COUNT
FLUSS_SERVER_COUNT="$(echo "$FLUSS_SERVER" | tr -s ':' '\n' | grep -v '^$'
| wc -l)"
diff --git a/fluss-lake/fluss-lake-iceberg/pom.xml
b/fluss-lake/fluss-lake-iceberg/pom.xml
index 46fcbab1f..195ec8d90 100644
--- a/fluss-lake/fluss-lake-iceberg/pom.xml
+++ b/fluss-lake/fluss-lake-iceberg/pom.xml
@@ -115,6 +115,12 @@
<artifactId>hadoop-mapreduce-client-core</artifactId>
<version>2.8.5</version>
<scope>test</scope>
+ <exclusions>
+ <exclusion>
+ <artifactId>commons-io</artifactId>
+ <groupId>commons-io</groupId>
+ </exclusion>
+ </exclusions>
</dependency>
<dependency>
diff --git a/fluss-lake/fluss-lake-paimon/pom.xml
b/fluss-lake/fluss-lake-paimon/pom.xml
index c1ee02f42..105816a96 100644
--- a/fluss-lake/fluss-lake-paimon/pom.xml
+++ b/fluss-lake/fluss-lake-paimon/pom.xml
@@ -81,6 +81,12 @@
<artifactId>hadoop-mapreduce-client-core</artifactId>
<version>2.8.5</version>
<scope>test</scope>
+ <exclusions>
+ <exclusion>
+ <artifactId>commons-io</artifactId>
+ <groupId>commons-io</groupId>
+ </exclusion>
+ </exclusions>
</dependency>
<dependency>
diff --git a/fluss-server/pom.xml b/fluss-server/pom.xml
index 5569b4649..76b31f6e7 100644
--- a/fluss-server/pom.xml
+++ b/fluss-server/pom.xml
@@ -133,6 +133,12 @@
<include>*:*</include>
</includes>
</artifactSet>
+ <relocations>
+ <relocation>
+ <pattern>org.apache.commons</pattern>
+
<shadedPattern>org.apache.fluss.shaded.org.apache.commons</shadedPattern>
+ </relocation>
+ </relocations>
</configuration>
</execution>
</executions>
diff --git a/website/docs/maintenance/filesystems/hdfs.md
b/website/docs/maintenance/filesystems/hdfs.md
index b6ec25eaa..b7144cf32 100644
--- a/website/docs/maintenance/filesystems/hdfs.md
+++ b/website/docs/maintenance/filesystems/hdfs.md
@@ -53,3 +53,17 @@ fluss.hadoop.dfs.web.authentication.kerberos.principal:
HTTP/[email protected]
# Client principal and keytab (adjust paths as needed)
fluss.hadoop.hadoop.security.kerberos.ticket.cache.path: /tmp/krb5cc_1000
```
+
+#### Use Machine Hadoop Environment Configuration
+
+Fluss includes bundled Hadoop libraries with version 3.3.4 for deploying Fluss
in machine without Hadoop installed.
+For most use cases, these work perfectly. However, you should configure your
machine's native Hadoop environment if:
+1. Your HDFS uses kerberos security
+2. You need to avoid version conflicts between Fluss's bundled hadoop
libraries and your HDFS cluster
+
+Fluss automatically loads HDFS dependencies on the machine via the
`HADOOP_CLASSPATH` environment variable.
+Make sure that the `HADOOP_CLASSPATH` environment variable is set up (it can
be checked by running `echo $HADOOP_CLASSPATH`).
+If not, set it up using
+```bash
+export HADOOP_CLASSPATH=`hadoop classpath`
+```
diff --git a/website/docs/maintenance/tiered-storage/lakehouse-storage.md
b/website/docs/maintenance/tiered-storage/lakehouse-storage.md
index 2b0cf0d10..bcce00cde 100644
--- a/website/docs/maintenance/tiered-storage/lakehouse-storage.md
+++ b/website/docs/maintenance/tiered-storage/lakehouse-storage.md
@@ -44,9 +44,14 @@ datalake.paimon.metastore: hive
datalake.paimon.uri: thrift://<hive-metastore-host-name>:<port>
datalake.paimon.warehouse: hdfs:///path/to/warehouse
```
+
#### Add other jars required by datalake
While Fluss includes the core Paimon library, additional jars may still need
to be manually added to `${FLUSS_HOME}/plugins/paimon/` according to your needs.
-For example, for OSS filesystem support, you need to put
`paimon-oss-<paimon_version>.jar` into directory
`${FLUSS_HOME}/plugins/paimon/`.
+For example:
+- If you are using Paimon filesystem catalog with OSS filesystem, you need to
put `paimon-oss-<paimon_version>.jar` into directory
`${FLUSS_HOME}/plugins/paimon/`.
+- If you are using Paimon Hive catalog, you need to put [the flink sql hive
connector
jar](https://nightlies.apache.org/flink/flink-docs-stable/docs/connectors/table/hive/overview/#using-bundled-hive-jar)
into directory `${FLUSS_HOME}/plugins/paimon/`.
+
+Additionally, when using Paimon with HDFS, you must also configure the Fluss
server with the Hadoop environment. See the [HDFS setup
guide](/docs/maintenance/filesystems/hdfs.md) for detailed instructions.
### Start The Datalake Tiering Service
Then, you must start the datalake tiering service to tier Fluss's data to the
lakehouse storage.