Skip to content

Commit 9f34534

Browse files
authored
[HUDI-8093] Replace Hadoop Configuration with StorageConfiguration in hudi-client-common module (apache#11792)
Co-authored-by: Shawn Chang <[email protected]>
1 parent e4e5bf0 commit 9f34534

File tree

27 files changed

+151
-151
lines changed

27 files changed

+151
-151
lines changed

hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java

+9-8
Original file line numberDiff line numberDiff line change
@@ -41,15 +41,14 @@
4141
import org.apache.hudi.exception.HoodieException;
4242
import org.apache.hudi.exception.HoodieIOException;
4343
import org.apache.hudi.exception.HoodieWriteConflictException;
44-
import org.apache.hudi.hadoop.fs.HadoopFSUtils;
4544
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
4645
import org.apache.hudi.metrics.HoodieMetrics;
4746
import org.apache.hudi.storage.HoodieStorage;
4847
import org.apache.hudi.storage.HoodieStorageUtils;
48+
import org.apache.hudi.storage.StorageConfiguration;
4949
import org.apache.hudi.table.HoodieTable;
5050

5151
import com.codahale.metrics.Timer;
52-
import org.apache.hadoop.conf.Configuration;
5352
import org.slf4j.Logger;
5453
import org.slf4j.LoggerFactory;
5554

@@ -70,7 +69,7 @@ public abstract class BaseHoodieClient implements Serializable, AutoCloseable {
7069
private static final long serialVersionUID = 1L;
7170
protected final transient HoodieStorage storage;
7271
protected final transient HoodieEngineContext context;
73-
protected final transient Configuration hadoopConf;
72+
protected final transient StorageConfiguration<?> storageConf;
7473
protected final transient HoodieMetrics metrics;
7574
protected final HoodieWriteConfig config;
7675
protected final String basePath;
@@ -92,8 +91,8 @@ protected BaseHoodieClient(HoodieEngineContext context, HoodieWriteConfig client
9291

9392
protected BaseHoodieClient(HoodieEngineContext context, HoodieWriteConfig clientConfig,
9493
Option<EmbeddedTimelineService> timelineServer) {
95-
this.hadoopConf = context.getStorageConf().unwrapAs((Configuration.class));
96-
this.storage = HoodieStorageUtils.getStorage(clientConfig.getBasePath(), HadoopFSUtils.getStorageConf(hadoopConf));
94+
this.storageConf = context.getStorageConf();
95+
this.storage = HoodieStorageUtils.getStorage(clientConfig.getBasePath(), storageConf);
9796
this.context = context;
9897
this.basePath = clientConfig.getBasePath();
9998
this.config = clientConfig;
@@ -105,7 +104,7 @@ protected BaseHoodieClient(HoodieEngineContext context, HoodieWriteConfig client
105104
this.metrics = new HoodieMetrics(config, storage);
106105
this.txnManager = new TransactionManager(config, storage);
107106
this.timeGenerator = TimeGenerators.getTimeGenerator(
108-
config.getTimeGeneratorConfig(), HadoopFSUtils.getStorageConf(hadoopConf));
107+
config.getTimeGeneratorConfig(), storageConf);
109108
startEmbeddedServerView();
110109
initWrapperFSMetrics();
111110
runClientInitCallbacks();
@@ -183,8 +182,10 @@ protected void initWrapperFSMetrics() {
183182

184183
protected HoodieTableMetaClient createMetaClient(boolean loadActiveTimelineOnLoad) {
185184
return HoodieTableMetaClient.builder()
186-
.setConf(HadoopFSUtils.getStorageConfWithCopy(hadoopConf)).setBasePath(config.getBasePath())
187-
.setLoadActiveTimelineOnLoad(loadActiveTimelineOnLoad).setConsistencyGuardConfig(config.getConsistencyGuardConfig())
185+
.setConf(storageConf.newInstance())
186+
.setBasePath(config.getBasePath())
187+
.setLoadActiveTimelineOnLoad(loadActiveTimelineOnLoad)
188+
.setConsistencyGuardConfig(config.getConsistencyGuardConfig())
188189
.setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion())))
189190
.setTimeGeneratorConfig(config.getTimeGeneratorConfig())
190191
.setFileSystemRetryConfig(config.getFileSystemRetryConfig())

hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java

+15-15
Original file line numberDiff line numberDiff line change
@@ -59,14 +59,14 @@
5959
import org.apache.hudi.exception.HoodieRollbackException;
6060
import org.apache.hudi.metadata.HoodieTableMetadata;
6161
import org.apache.hudi.metadata.HoodieTableMetadataUtil;
62+
import org.apache.hudi.storage.StorageConfiguration;
6263
import org.apache.hudi.table.HoodieTable;
6364
import org.apache.hudi.table.action.HoodieWriteMetadata;
6465
import org.apache.hudi.table.action.compact.CompactHelpers;
6566
import org.apache.hudi.table.action.rollback.RollbackUtils;
6667
import org.apache.hudi.table.marker.WriteMarkersFactory;
6768

6869
import com.codahale.metrics.Timer;
69-
import org.apache.hadoop.conf.Configuration;
7070
import org.slf4j.Logger;
7171
import org.slf4j.LoggerFactory;
7272

@@ -165,7 +165,7 @@ protected void setPendingInflightAndRequestedInstants(Set<String> pendingInfligh
165165
protected void preCommit(HoodieCommitMetadata metadata) {
166166
// Create a Hoodie table after startTxn which encapsulated the commits and files visible.
167167
// Important to create this after the lock to ensure the latest commits show up in the timeline without need for reload
168-
HoodieTable table = createTable(config, hadoopConf);
168+
HoodieTable table = createTable(config, storageConf);
169169
resolveWriteConflict(table, metadata, this.pendingInflightAndRequestedInstants);
170170
}
171171

@@ -198,7 +198,7 @@ private void inlineCompaction(HoodieTable table, Option<Map<String, String>> ext
198198
* @return Collection of Write Status
199199
*/
200200
protected HoodieWriteMetadata<O> logCompact(String logCompactionInstantTime, boolean shouldComplete) {
201-
HoodieTable<?, I, ?, T> table = createTable(config, context.getStorageConf().unwrapAs(Configuration.class));
201+
HoodieTable<?, I, ?, T> table = createTable(config, context.getStorageConf());
202202

203203
// Check if a commit or compaction instant with a greater timestamp is on the timeline.
204204
// If an instant is found then abort log compaction, since it is no longer needed.
@@ -288,7 +288,7 @@ public Option<String> scheduleCompaction(Option<Map<String, String>> extraMetada
288288
* @return Collection of Write Status
289289
*/
290290
protected HoodieWriteMetadata<O> compact(String compactionInstantTime, boolean shouldComplete) {
291-
HoodieTable<?, I, ?, T> table = createTable(config, context.getStorageConf().unwrapAs(Configuration.class));
291+
HoodieTable<?, I, ?, T> table = createTable(config, context.getStorageConf());
292292
HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline();
293293
HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime);
294294
if (pendingCompactionTimeline.containsInstant(inflightInstant)) {
@@ -313,7 +313,7 @@ protected HoodieWriteMetadata<O> compact(String compactionInstantTime, boolean s
313313
*/
314314
public void commitCompaction(String compactionInstantTime, HoodieCommitMetadata metadata, Option<Map<String, String>> extraMetadata) {
315315
extraMetadata.ifPresent(m -> m.forEach(metadata::addMetadata));
316-
completeCompaction(metadata, createTable(config, context.getStorageConf().unwrapAs(Configuration.class)), compactionInstantTime);
316+
completeCompaction(metadata, createTable(config, context.getStorageConf()), compactionInstantTime);
317317
}
318318

319319
/**
@@ -446,7 +446,7 @@ public boolean scheduleClusteringAtInstant(String instantTime, Option<Map<String
446446
* @return Collection of Write Status
447447
*/
448448
public HoodieWriteMetadata<O> cluster(String clusteringInstant, boolean shouldComplete) {
449-
HoodieTable<?, I, ?, T> table = createTable(config, context.getStorageConf().unwrapAs(Configuration.class));
449+
HoodieTable<?, I, ?, T> table = createTable(config, context.getStorageConf());
450450
HoodieTimeline pendingClusteringTimeline = table.getActiveTimeline().filterPendingReplaceOrClusteringTimeline();
451451
Option<HoodieInstant> inflightInstantOpt = ClusteringUtils.getInflightClusteringInstant(clusteringInstant, table.getActiveTimeline());
452452
if (inflightInstantOpt.isPresent()) {
@@ -481,7 +481,7 @@ public HoodieWriteMetadata<O> cluster(String clusteringInstant, boolean shouldCo
481481
}
482482

483483
public boolean purgePendingClustering(String clusteringInstant) {
484-
HoodieTable<?, I, ?, T> table = createTable(config, context.getStorageConf().unwrapAs(Configuration.class));
484+
HoodieTable<?, I, ?, T> table = createTable(config, context.getStorageConf());
485485
Option<HoodieInstant> inflightInstantOpt = ClusteringUtils.getInflightClusteringInstant(clusteringInstant, table.getActiveTimeline());
486486
if (inflightInstantOpt.isPresent()) {
487487
table.rollbackInflightClustering(inflightInstantOpt.get(), commitToRollback -> getPendingRollbackInfo(table.getMetaClient(), commitToRollback, false), true);
@@ -498,7 +498,7 @@ public boolean purgePendingClustering(String clusteringInstant) {
498498
* @return HoodieWriteMetadata
499499
*/
500500
public HoodieWriteMetadata<T> managePartitionTTL(String instantTime) {
501-
HoodieTable<?, I, ?, T> table = createTable(config, context.getStorageConf().unwrapAs(Configuration.class));
501+
HoodieTable<?, I, ?, T> table = createTable(config, context.getStorageConf());
502502
return table.managePartitionTTL(context, instantTime);
503503
}
504504

@@ -633,7 +633,7 @@ protected Option<String> scheduleTableServiceInternal(String instantTime, Option
633633
}
634634

635635
Option<String> option = Option.empty();
636-
HoodieTable<?, ?, ?, ?> table = createTable(config, hadoopConf);
636+
HoodieTable<?, ?, ?, ?> table = createTable(config, storageConf);
637637

638638
switch (tableServiceType) {
639639
case ARCHIVE:
@@ -675,7 +675,7 @@ protected Option<String> scheduleTableServiceInternal(String instantTime, Option
675675
return option;
676676
}
677677

678-
protected abstract HoodieTable<?, I, ?, T> createTable(HoodieWriteConfig config, Configuration hadoopConf);
678+
protected abstract HoodieTable<?, I, ?, T> createTable(HoodieWriteConfig config, StorageConfiguration<?> storageConf);
679679

680680
/**
681681
* Executes a clustering plan on a table, serially before or after an insert/upsert action.
@@ -754,7 +754,7 @@ public HoodieCleanMetadata clean(String cleanInstantTime, boolean scheduleInline
754754
CleanerUtils.rollbackFailedWrites(config.getFailedWritesCleanPolicy(),
755755
HoodieTimeline.CLEAN_ACTION, () -> rollbackFailedWrites());
756756

757-
HoodieTable table = createTable(config, hadoopConf);
757+
HoodieTable table = createTable(config, storageConf);
758758
if (config.allowMultipleCleans() || !table.getActiveTimeline().getCleanerTimeline().filterInflightsAndRequested().firstInstant().isPresent()) {
759759
LOG.info("Cleaner started");
760760
// proceed only if multiple clean schedules are enabled or if there are no pending cleans.
@@ -892,7 +892,7 @@ protected Map<String, Option<HoodiePendingRollbackInfo>> getPendingRollbackInfos
892892
* @return {@code true} if rollback happens; {@code false} otherwise.
893893
*/
894894
protected boolean rollbackFailedIndexingCommits() {
895-
HoodieTable table = createTable(config, hadoopConf);
895+
HoodieTable table = createTable(config, storageConf);
896896
List<String> instantsToRollback = getFailedIndexingCommitsToRollbackForMetadataTable(table.getMetaClient());
897897
Map<String, Option<HoodiePendingRollbackInfo>> pendingRollbacks = getPendingRollbackInfos(table.getMetaClient());
898898
instantsToRollback.forEach(entry -> pendingRollbacks.putIfAbsent(entry, Option.empty()));
@@ -929,7 +929,7 @@ && isIndexingCommit(dataIndexTimeline, instant.getTimestamp()))
929929
* @return true if rollback was triggered. false otherwise.
930930
*/
931931
protected Boolean rollbackFailedWrites() {
932-
HoodieTable table = createTable(config, hadoopConf);
932+
HoodieTable table = createTable(config, storageConf);
933933
List<String> instantsToRollback = getInstantsToRollback(table.getMetaClient(), config.getFailedWritesCleanPolicy(), Option.empty());
934934
Map<String, Option<HoodiePendingRollbackInfo>> pendingRollbacks = getPendingRollbackInfos(table.getMetaClient());
935935
instantsToRollback.forEach(entry -> pendingRollbacks.putIfAbsent(entry, Option.empty()));
@@ -1051,7 +1051,7 @@ public boolean rollback(final String commitInstantTime, Option<HoodiePendingRoll
10511051
LOG.info("Begin rollback of instant " + commitInstantTime);
10521052
final Timer.Context timerContext = this.metrics.getRollbackCtx();
10531053
try {
1054-
HoodieTable table = createTable(config, hadoopConf);
1054+
HoodieTable table = createTable(config, storageConf);
10551055
Option<HoodieInstant> commitInstantOpt = Option.fromJavaOptional(table.getActiveTimeline().getCommitsTimeline().getInstantsAsStream()
10561056
.filter(instant -> HoodieActiveTimeline.EQUALS.test(instant.getTimestamp(), commitInstantTime))
10571057
.findFirst());
@@ -1096,7 +1096,7 @@ public boolean rollback(final String commitInstantTime, Option<HoodiePendingRoll
10961096
*/
10971097
public void rollbackFailedBootstrap() {
10981098
LOG.info("Rolling back pending bootstrap if present");
1099-
HoodieTable table = createTable(config, hadoopConf);
1099+
HoodieTable table = createTable(config, storageConf);
11001100
HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingMajorAndMinorCompaction();
11011101
Option<String> instant = Option.fromJavaOptional(
11021102
inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp).findFirst());

hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java

+1-2
Original file line numberDiff line numberDiff line change
@@ -64,7 +64,6 @@
6464
import org.apache.hudi.exception.HoodieRestoreException;
6565
import org.apache.hudi.exception.HoodieRollbackException;
6666
import org.apache.hudi.exception.HoodieSavepointException;
67-
import org.apache.hudi.hadoop.fs.HadoopFSUtils;
6867
import org.apache.hudi.index.HoodieIndex;
6968
import org.apache.hudi.internal.schema.InternalSchema;
7069
import org.apache.hudi.internal.schema.Type;
@@ -697,7 +696,7 @@ public void restoreToSavepoint(String savepointTime) {
697696
// or before the oldest compaction on MDT.
698697
// We cannot restore to before the oldest compaction on MDT as we don't have the basefiles before that time.
699698
HoodieTableMetaClient mdtMetaClient = HoodieTableMetaClient.builder()
700-
.setConf(HadoopFSUtils.getStorageConfWithCopy(hadoopConf))
699+
.setConf(storageConf.newInstance())
701700
.setBasePath(getMetadataTableBasePath(config.getBasePath())).build();
702701
Option<HoodieInstant> oldestMdtCompaction = mdtMetaClient.getCommitTimeline().filterCompletedInstants().firstInstant();
703702
boolean deleteMDT = false;

hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java

+2-3
Original file line numberDiff line numberDiff line change
@@ -31,7 +31,6 @@
3131
import org.apache.hudi.storage.StorageConfiguration;
3232
import org.apache.hudi.timeline.service.TimelineService;
3333

34-
import org.apache.hadoop.conf.Configuration;
3534
import org.slf4j.Logger;
3635
import org.slf4j.LoggerFactory;
3736

@@ -181,15 +180,15 @@ private void startServer(TimelineServiceCreator timelineServiceCreator) throws I
181180

182181
this.serviceConfig = timelineServiceConfBuilder.build();
183182

184-
server = timelineServiceCreator.create(context, storageConf.unwrapCopyAs(Configuration.class), serviceConfig,
183+
server = timelineServiceCreator.create(context, storageConf.newInstance(), serviceConfig,
185184
HoodieStorageUtils.getStorage(writeConfig.getBasePath(), storageConf.newInstance()), viewManager);
186185
serverPort = server.startService();
187186
LOG.info("Started embedded timeline server at " + hostAddr + ":" + serverPort);
188187
}
189188

190189
@FunctionalInterface
191190
interface TimelineServiceCreator {
192-
TimelineService create(HoodieEngineContext context, Configuration hadoopConf, TimelineService.Config timelineServerConf,
191+
TimelineService create(HoodieEngineContext context, StorageConfiguration<?> storageConf, TimelineService.Config timelineServerConf,
193192
HoodieStorage storage, FileSystemViewManager globalFileSystemViewManager) throws IOException;
194193
}
195194

hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java

+3-4
Original file line numberDiff line numberDiff line change
@@ -29,17 +29,16 @@
2929
import org.apache.hudi.config.HoodieWriteConfig;
3030
import org.apache.hudi.exception.HoodieException;
3131
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
32+
import org.apache.hudi.storage.StorageConfiguration;
3233
import org.apache.hudi.table.HoodieTable;
3334

34-
import org.apache.hadoop.conf.Configuration;
35-
3635
import java.io.Serializable;
3736

3837
public abstract class BaseActionExecutor<T, I, K, O, R> implements Serializable {
3938

4039
private static final long serialVersionUID = 1L;
4140
protected final transient HoodieEngineContext context;
42-
protected final transient Configuration hadoopConf;
41+
protected final transient StorageConfiguration<?> storageConf;
4342

4443
protected final HoodieWriteConfig config;
4544

@@ -49,7 +48,7 @@ public abstract class BaseActionExecutor<T, I, K, O, R> implements Serializable
4948

5049
public BaseActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable<T, I, K, O> table, String instantTime) {
5150
this.context = context;
52-
this.hadoopConf = context.getStorageConf().unwrapAs(Configuration.class);
51+
this.storageConf = context.getStorageConf();
5352
this.config = config;
5453
this.table = table;
5554
this.instantTime = instantTime;

hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapUtils.java

+1-2
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,6 @@
2525
import org.apache.hudi.common.util.collection.Pair;
2626
import org.apache.hudi.hadoop.fs.HadoopFSUtils;
2727

28-
import org.apache.hadoop.conf.Configuration;
2928
import org.apache.hadoop.fs.FileStatus;
3029
import org.apache.hadoop.fs.FileSystem;
3130
import org.apache.hadoop.fs.LocatedFileStatus;
@@ -80,7 +79,7 @@ public static List<Pair<String, List<HoodieFileStatus>>> getAllLeafFoldersWithFi
8079
result.addAll(context.flatMap(subDirectories, directory -> {
8180
PathFilter pathFilter = getFilePathFilter(baseFileExtension);
8281
Path path = new Path(directory);
83-
FileSystem fileSystem = path.getFileSystem(new Configuration());
82+
FileSystem fileSystem = HadoopFSUtils.getFs(path, HadoopFSUtils.getStorageConf());
8483
RemoteIterator<LocatedFileStatus> itr = fileSystem.listFiles(path, true);
8584
List<Pair<HoodieFileStatus, Pair<Integer, String>>> res = new ArrayList<>();
8685
while (itr.hasNext()) {

hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java

+1-2
Original file line numberDiff line numberDiff line change
@@ -36,7 +36,6 @@
3636
import org.apache.hudi.config.HoodieWriteConfig;
3737
import org.apache.hudi.exception.HoodieIndexException;
3838
import org.apache.hudi.exception.HoodieMetadataException;
39-
import org.apache.hudi.hadoop.fs.HadoopFSUtils;
4039
import org.apache.hudi.metadata.HoodieMetadataMetrics;
4140
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
4241
import org.apache.hudi.metadata.MetadataPartitionType;
@@ -158,7 +157,7 @@ public Option<HoodieIndexCommitMetadata> execute() {
158157
// reconcile with metadata table timeline
159158
String metadataBasePath = getMetadataTableBasePath(table.getMetaClient().getBasePath().toString());
160159
HoodieTableMetaClient metadataMetaClient = HoodieTableMetaClient.builder()
161-
.setConf(HadoopFSUtils.getStorageConfWithCopy(hadoopConf))
160+
.setConf(storageConf.newInstance())
162161
.setBasePath(metadataBasePath).build();
163162
Set<String> metadataCompletedTimestamps = getCompletedArchivedAndActiveInstantsAfter(indexUptoInstant, metadataMetaClient).stream()
164163
.map(HoodieInstant::getTimestamp).collect(Collectors.toSet());

hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java

+1-2
Original file line numberDiff line numberDiff line change
@@ -39,7 +39,6 @@
3939
import org.apache.hudi.storage.StoragePathInfo;
4040
import org.apache.hudi.table.HoodieTable;
4141

42-
import org.apache.hadoop.conf.Configuration;
4342
import org.apache.hadoop.fs.FileStatus;
4443
import org.apache.hadoop.fs.FileSystem;
4544
import org.apache.hadoop.fs.LocatedFileStatus;
@@ -118,7 +117,7 @@ public Set<String> createdAndMergedDataPaths(HoodieEngineContext context, int pa
118117
context.setJobStatus(this.getClass().getSimpleName(), "Obtaining marker files for all created, merged paths");
119118
dataFiles.addAll(context.flatMap(subDirectories, directory -> {
120119
Path path = new Path(directory);
121-
FileSystem fileSystem = HadoopFSUtils.getFs(path, storageConf.unwrapAs(Configuration.class));
120+
FileSystem fileSystem = HadoopFSUtils.getFs(path, storageConf);
122121
RemoteIterator<LocatedFileStatus> itr = fileSystem.listFiles(path, true);
123122
List<String> result = new ArrayList<>();
124123
while (itr.hasNext()) {

hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkersFactory.java

+1-2
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,6 @@
2424
import org.apache.hudi.storage.StorageSchemes;
2525
import org.apache.hudi.table.HoodieTable;
2626

27-
import org.apache.hadoop.conf.Configuration;
2827
import org.slf4j.Logger;
2928
import org.slf4j.LoggerFactory;
3029

@@ -53,7 +52,7 @@ public static WriteMarkers get(MarkerType markerType, HoodieTable table, String
5352
}
5453
String basePath = table.getMetaClient().getBasePath().toString();
5554
if (StorageSchemes.HDFS.getScheme().equals(
56-
HadoopFSUtils.getFs(basePath, table.getContext().getStorageConf().unwrapCopyAs(Configuration.class)).getScheme())) {
55+
HadoopFSUtils.getFs(basePath, table.getContext().getStorageConf(), true).getScheme())) {
5756
LOG.warn("Timeline-server-based markers are not supported for HDFS: "
5857
+ "base path " + basePath + ". Falling back to direct markers.");
5958
return new DirectWriteMarkers(table, instantTime);

0 commit comments

Comments
 (0)