-
Notifications
You must be signed in to change notification settings - Fork 1.2k
Description
Description
CatalogContext currently has a hard dependency on Hadoop Configuration, causing NoClassDefFoundError in environments where Hadoop is not needed or available.
Critical: Trino Plugin Development Blocker
This issue is a critical blocker for developing the Trino-Paimon connector.
Trino's Mandatory Requirement
Trino explicitly prohibits connectors from having mandatory Hadoop dependencies:
🔗 Trino Policy: trinodb/trino#15921
Quote from Trino maintainers:
"Trino connectors should not have a hard dependency on Hadoop. Connectors must work without Hadoop on the classpath."
Previous Paimon-Trino Implementation Issue
The previous paimon-trino implementation was affected by this problem:
🔗 Paimon-Trino Issue: apache/paimon-trino#96
This issue prevented proper deployment and usage of Paimon with Trino in production environments.
Use Cases Affected
1. 🎯 Trino-Paimon Connector (PRIMARY)
- Status: Currently BLOCKED by mandatory Hadoop dependency
- Requirement: Trino's connector architecture requires optional Hadoop support
- Impact: Cannot integrate Paimon with Trino's cloud-native deployment model
- Policy: Decouple Trino from Hadoop and Hive codebases trinodb/trino#15921
- Previous Issue: paimon-trino-440使用异常 paimon-trino#96
2. Windows Development Environment
When using Paimon on Windows (e.g., Flink CDC with Paimon sink to MinIO S3), the application fails with:
Caused by: java.lang.NoClassDefFoundError: org/apache/hadoop/conf/Configuration
at org.apache.paimon.catalog.CatalogContext.<init>(CatalogContext.java:53)
at org.apache.paimon.catalog.CatalogContext.create(CatalogContext.java:73)
at org.apache.paimon.flink.FlinkCatalogFactory.createPaimonCatalog(FlinkCatalogFactory.java:81)
at org.apache.flink.cdc.connectors.paimon.sink.v2.bucket.BucketAssignOperator.open(BucketAssignOperator.java:103)
at org.apache.flink.streaming.runtime.tasks.RegularOperatorChain.initializeStateAndOpenOperators(RegularOperatorChain.java:107)
at org.apache.flink.streaming.runtime.tasks.StreamTask.restoreStateAndGates(StreamTask.java:858)
at org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$restoreInternal$5(StreamTask.java:812)
at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.call(StreamTaskActionExecutor.java:55)
at org.apache.flink.streaming.runtime.tasks.StreamTask.restoreInternal(StreamTask.java:812)
at org.apache.flink.streaming.runtime.tasks.StreamTask.restore(StreamTask.java:771)
at org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:970)
at org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:939)
at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:763)
at org.apache.flink.runtime.taskmanager.Task.run(Task.java:575)
at java.base/java.lang.Thread.run(Thread.java:842)
Caused by: java.lang.ClassNotFoundException: org.apache.hadoop.conf.Configuration
3. Lightweight Deployment Scenarios
- Local FileIO usage shouldn't require Hadoop
- Cloud-native deployments using native S3/OSS clients
- Embedded/lightweight environments
- Container deployments without Hadoop ecosystem
Problem Root Cause
The current CatalogContext implementation has a mandatory Hadoop dependency:
public class CatalogContext {
private final Configuration hadoopConf; // Always requires Hadoop
private CatalogContext(..., Configuration hadoopConf) {
this.hadoopConf = hadoopConf; // Hard dependency - blocks Trino
}
public Configuration hadoopConf() {
return hadoopConf; // Forces ClassNotFoundException without Hadoop
}
}This design:
- ❌ Violates Trino's connector design principles
- ❌ Prevents deployment in Hadoop-free environments
- ❌ Increases dependency footprint unnecessarily
- ❌ Blocks Paimon adoption in cloud-native scenarios
Proposed Solution
Refactor CatalogContext using class hierarchy to separate concerns:
1. CatalogContext - Base class (Hadoop-free)
public class CatalogContext {
protected final Options options;
protected final ClassLoader classLoader;
// No Hadoop dependency
}2. HadoopAware - Interface for Hadoop functionality
public interface HadoopAware {
Configuration hadoopConf();
}3. CatalogHadoopContext - Hadoop implementation
public class CatalogHadoopContext extends CatalogContext implements HadoopAware {
private final Configuration hadoopConf;
@Override
public Configuration hadoopConf() {
return hadoopConf;
}
}Components Update
- FileIO: Use
CatalogContextas base type - Runtime check: Components check for
HadoopAwareinterface when Hadoop is needed - Graceful fallback: Works without Hadoop, enables Hadoop when available
- Factory pattern: Automatically returns appropriate type based on environment
Expected Benefits
- ✅ Unblocks Trino-Paimon connector - Complies with Trino's requirements
- ✅ Fixes Windows development with Flink CDC + Paimon + MinIO
- ✅ Reduces dependency footprint for cloud-native deployments
- ✅ Improves architecture (separation of concerns)
- ✅ Maintains backward compatibility - existing code continues to work
Impact Assessment
- Critical: Trino connector development (currently blocked)
- High: Windows development environments
- Medium: Lightweight deployments
- Low: Existing Flink/Spark usage (backward compatible)