1
0

[HUDI-137] Hudi cleaning state changes should be consistent with compaction actions

Before this change, Cleaner performs cleaning of old file versions and then stores the deleted files in .clean files.
With this setup, we will not be able to track file deletions if a cleaner fails after deleting files but before writing .clean metadata.
This is fine for regular file-system view generation but Incremental timeline syncing relies on clean/commit/compaction metadata to keep a consistent file-system view.

Cleaner state transitions is now similar to that of compaction.

1. Requested : HoodieWriteClient.scheduleClean() selects the list of files that needs to be deleted and stores them in metadata
2. Inflight : HoodieWriteClient marks the state to be inflight before it starts deleting
3. Completed : HoodieWriteClient marks the state after completing the deletion according to the cleaner plan
This commit is contained in:
Balaji Varadarajan
2019-10-28 18:54:48 -07:00
committed by Balaji Varadarajan
parent 23b303e4b1
commit 1032fc3e54
34 changed files with 856 additions and 491 deletions

View File

@@ -0,0 +1,69 @@
/*
* 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.
*/
{
"namespace": "org.apache.hudi.avro.model",
"type": "record",
"name": "HoodieCleanerPlan",
"fields": [
{
"name": "earliestInstantToRetain",
"type":["null", {
"type": "record",
"name": "HoodieActionInstant",
"fields": [
{
"name": "timestamp",
"type": "string"
},
{
"name": "action",
"type": "string"
},
{
"name": "state",
"type": "string"
}
]
}],
"default" : null
},
{
"name": "policy",
"type": "string"
},
{
"name": "filesToBeDeletedPerPartition",
"type": [
"null", {
"type":"map",
"values": {
"type":"array",
"items":{
"name":"filePath",
"type": "string"
}
}}],
"default" : null
},
{
"name":"version",
"type":["int", "null"],
"default": 1
}
]
}

View File

@@ -134,14 +134,9 @@ public class CompactionOperation implements Serializable {
@Override
public String toString() {
return "CompactionOperation{"
+ "baseInstantTime='" + baseInstantTime + '\''
+ ", dataFileCommitTime=" + dataFileCommitTime
+ ", deltaFileNames=" + deltaFileNames
+ ", dataFileName=" + dataFileName
+ ", id='" + id + '\''
+ ", metrics=" + metrics
+ '}';
return "CompactionOperation{" + "baseInstantTime='" + baseInstantTime + '\'' + ", dataFileCommitTime="
+ dataFileCommitTime + ", deltaFileNames=" + deltaFileNames + ", dataFileName=" + dataFileName + ", id='" + id
+ '\'' + ", metrics=" + metrics + '}';
}
@Override
@@ -156,8 +151,7 @@ public class CompactionOperation implements Serializable {
return Objects.equals(baseInstantTime, operation.baseInstantTime)
&& Objects.equals(dataFileCommitTime, operation.dataFileCommitTime)
&& Objects.equals(deltaFileNames, operation.deltaFileNames)
&& Objects.equals(dataFileName, operation.dataFileName)
&& Objects.equals(id, operation.id);
&& Objects.equals(dataFileName, operation.dataFileName) && Objects.equals(id, operation.id);
}
@Override

View File

@@ -63,6 +63,7 @@ public interface HoodieTimeline extends Serializable {
String INFLIGHT_COMMIT_EXTENSION = INFLIGHT_EXTENSION;
String INFLIGHT_DELTA_COMMIT_EXTENSION = "." + DELTA_COMMIT_ACTION + INFLIGHT_EXTENSION;
String INFLIGHT_CLEAN_EXTENSION = "." + CLEAN_ACTION + INFLIGHT_EXTENSION;
String REQUESTED_CLEAN_EXTENSION = "." + CLEAN_ACTION + REQUESTED_EXTENSION;
String INFLIGHT_ROLLBACK_EXTENSION = "." + ROLLBACK_ACTION + INFLIGHT_EXTENSION;
String INFLIGHT_SAVEPOINT_EXTENSION = "." + SAVEPOINT_ACTION + INFLIGHT_EXTENSION;
String REQUESTED_COMPACTION_SUFFIX = StringUtils.join(COMPACTION_ACTION, REQUESTED_EXTENSION);
@@ -80,6 +81,13 @@ public interface HoodieTimeline extends Serializable {
*/
HoodieTimeline filterInflights();
/**
* Filter this timeline to include requested and in-flights
*
* @return New instance of HoodieTimeline with just in-flights and requested instants
*/
HoodieTimeline filterInflightsAndRequested();
/**
* Filter this timeline to just include the in-flights excluding compaction instants
*
@@ -219,7 +227,19 @@ public interface HoodieTimeline extends Serializable {
}
static HoodieInstant getCompletedInstant(final HoodieInstant instant) {
return new HoodieInstant(false, instant.getAction(), instant.getTimestamp());
return new HoodieInstant(State.COMPLETED, instant.getAction(), instant.getTimestamp());
}
static HoodieInstant getRequestedInstant(final HoodieInstant instant) {
return new HoodieInstant(State.REQUESTED, instant.getAction(), instant.getTimestamp());
}
static HoodieInstant getCleanRequestedInstant(final String timestamp) {
return new HoodieInstant(State.REQUESTED, CLEAN_ACTION, timestamp);
}
static HoodieInstant getCleanInflightInstant(final String timestamp) {
return new HoodieInstant(State.INFLIGHT, CLEAN_ACTION, timestamp);
}
static HoodieInstant getCompactionRequestedInstant(final String timestamp) {
@@ -246,6 +266,10 @@ public interface HoodieTimeline extends Serializable {
return StringUtils.join(instant, HoodieTimeline.CLEAN_EXTENSION);
}
static String makeRequestedCleanerFileName(String instant) {
return StringUtils.join(instant, HoodieTimeline.REQUESTED_CLEAN_EXTENSION);
}
static String makeInflightCleanerFileName(String instant) {
return StringUtils.join(instant, HoodieTimeline.INFLIGHT_CLEAN_EXTENSION);
}

View File

@@ -58,10 +58,11 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
public static final SimpleDateFormat COMMIT_FORMATTER = new SimpleDateFormat("yyyyMMddHHmmss");
public static final Set<String> VALID_EXTENSIONS_IN_ACTIVE_TIMELINE = new HashSet<>(Arrays.asList(new String[] {
COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION, INFLIGHT_DELTA_COMMIT_EXTENSION,
SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION, CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION,
INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION}));
public static final Set<String> VALID_EXTENSIONS_IN_ACTIVE_TIMELINE =
new HashSet<>(Arrays.asList(new String[] {COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION,
INFLIGHT_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION, CLEAN_EXTENSION,
INFLIGHT_CLEAN_EXTENSION, REQUESTED_CLEAN_EXTENSION, INFLIGHT_COMPACTION_EXTENSION,
REQUESTED_COMPACTION_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION}));
private static final transient Logger log = LogManager.getLogger(HoodieActiveTimeline.class);
protected HoodieTableMetaClient metaClient;
@@ -212,11 +213,19 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
}
public void revertToInflight(HoodieInstant instant) {
log.info("Reverting instant to inflight " + instant);
revertCompleteToInflight(instant, HoodieTimeline.getInflightInstant(instant));
log.info("Reverting " + instant + " to inflight ");
revertStateTransition(instant, HoodieTimeline.getInflightInstant(instant));
log.info("Reverted " + instant + " to inflight");
}
public HoodieInstant revertToRequested(HoodieInstant instant) {
log.warn("Reverting " + instant + " to requested ");
HoodieInstant requestedInstant = HoodieTimeline.getRequestedInstant(instant);
revertStateTransition(instant, HoodieTimeline.getRequestedInstant(instant));
log.warn("Reverted " + instant + " to requested");
return requestedInstant;
}
public void deleteInflight(HoodieInstant instant) {
Preconditions.checkArgument(instant.isInflight());
deleteInstantFile(instant);
@@ -311,6 +320,39 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
* END - COMPACTION RELATED META-DATA MANAGEMENT
**/
/**
* Transition Clean State from inflight to Committed
*
* @param inflightInstant Inflight instant
* @param data Extra Metadata
* @return commit instant
*/
public HoodieInstant transitionCleanInflightToComplete(HoodieInstant inflightInstant, Option<byte[]> data) {
Preconditions.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.CLEAN_ACTION));
Preconditions.checkArgument(inflightInstant.isInflight());
HoodieInstant commitInstant = new HoodieInstant(State.COMPLETED, CLEAN_ACTION, inflightInstant.getTimestamp());
// First write metadata to aux folder
createFileInAuxiliaryFolder(commitInstant, data);
// Then write to timeline
transitionState(inflightInstant, commitInstant, data);
return commitInstant;
}
/**
* Transition Clean State from requested to inflight
*
* @param requestedInstant requested instant
* @return commit instant
*/
public HoodieInstant transitionCleanRequestedToInflight(HoodieInstant requestedInstant) {
Preconditions.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.CLEAN_ACTION));
Preconditions.checkArgument(requestedInstant.isRequested());
HoodieInstant inflight = new HoodieInstant(State.INFLIGHT, CLEAN_ACTION, requestedInstant.getTimestamp());
transitionState(requestedInstant, inflight, Option.empty());
return inflight;
}
private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, Option<byte[]> data) {
Preconditions.checkArgument(fromInstant.getTimestamp().equals(toInstant.getTimestamp()));
Path commitFilePath = new Path(metaClient.getMetaPath(), toInstant.getFileName());
@@ -327,19 +369,20 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
}
}
private void revertCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) {
Preconditions.checkArgument(completed.getTimestamp().equals(inflight.getTimestamp()));
Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), inflight.getFileName());
private void revertStateTransition(HoodieInstant curr, HoodieInstant revert) {
Preconditions.checkArgument(curr.getTimestamp().equals(revert.getTimestamp()));
Path revertFilePath = new Path(metaClient.getMetaPath(), revert.getFileName());
try {
if (!metaClient.getFs().exists(inFlightCommitFilePath)) {
Path commitFilePath = new Path(metaClient.getMetaPath(), completed.getFileName());
boolean success = metaClient.getFs().rename(commitFilePath, inFlightCommitFilePath);
if (!metaClient.getFs().exists(revertFilePath)) {
Path currFilePath = new Path(metaClient.getMetaPath(), curr.getFileName());
boolean success = metaClient.getFs().rename(currFilePath, revertFilePath);
if (!success) {
throw new HoodieIOException("Could not rename " + commitFilePath + " to " + inFlightCommitFilePath);
throw new HoodieIOException("Could not rename " + currFilePath + " to " + revertFilePath);
}
log.info("Renamed " + currFilePath + " to " + revertFilePath);
}
} catch (IOException e) {
throw new HoodieIOException("Could not complete revert " + completed, e);
throw new HoodieIOException("Could not complete revert " + curr, e);
}
}
@@ -355,6 +398,15 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
createFileInMetaPath(instant.getFileName(), content);
}
public void saveToCleanRequested(HoodieInstant instant, Option<byte[]> content) {
Preconditions.checkArgument(instant.getAction().equals(HoodieTimeline.CLEAN_ACTION));
Preconditions.checkArgument(instant.getState().equals(State.REQUESTED));
// Write workload to auxiliary folder
createFileInAuxiliaryFolder(instant, content);
// Plan is only stored in auxiliary folder
createFileInMetaPath(instant.getFileName(), Option.empty());
}
private void createFileInMetaPath(String filename, Option<byte[]> content) {
Path fullPath = new Path(metaClient.getMetaPath(), filename);
createFileInPath(fullPath, content);

View File

@@ -30,6 +30,7 @@ import java.util.function.Predicate;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.exception.HoodieException;
@@ -83,6 +84,13 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
return new HoodieDefaultTimeline(instants.stream().filter(HoodieInstant::isInflight), details);
}
@Override
public HoodieTimeline filterInflightsAndRequested() {
return new HoodieDefaultTimeline(
instants.stream().filter(i -> i.getState().equals(State.REQUESTED) || i.getState().equals(State.INFLIGHT)),
details);
}
@Override
public HoodieTimeline filterInflightsExcludingCompaction() {
return new HoodieDefaultTimeline(instants.stream().filter(instant -> {

View File

@@ -69,7 +69,7 @@ public class HoodieInstant implements Serializable {
} else if (action.contains(HoodieTimeline.INFLIGHT_EXTENSION)) {
state = State.INFLIGHT;
action = action.replace(HoodieTimeline.INFLIGHT_EXTENSION, "");
} else if (action.equals(HoodieTimeline.REQUESTED_COMPACTION_SUFFIX)) {
} else if (action.contains(HoodieTimeline.REQUESTED_EXTENSION)) {
state = State.REQUESTED;
action = action.replace(HoodieTimeline.REQUESTED_EXTENSION, "");
}
@@ -117,7 +117,8 @@ public class HoodieInstant implements Serializable {
: HoodieTimeline.makeCommitFileName(timestamp);
} else if (HoodieTimeline.CLEAN_ACTION.equals(action)) {
return isInflight() ? HoodieTimeline.makeInflightCleanerFileName(timestamp)
: HoodieTimeline.makeCleanerFileName(timestamp);
: isRequested() ? HoodieTimeline.makeRequestedCleanerFileName(timestamp)
: HoodieTimeline.makeCleanerFileName(timestamp);
} else if (HoodieTimeline.ROLLBACK_ACTION.equals(action)) {
return isInflight() ? HoodieTimeline.makeInflightRollbackFileName(timestamp)
: HoodieTimeline.makeRollbackFileName(timestamp);

View File

@@ -333,8 +333,10 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
@Override
public void close() {
log.info("Closing Rocksdb !!");
closed = true;
rocksDB.close();
log.info("Closed Rocksdb !!");
}
@Override

View File

@@ -36,6 +36,7 @@ import org.apache.avro.specific.SpecificDatumWriter;
import org.apache.avro.specific.SpecificRecordBase;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieCleanPartitionMetadata;
import org.apache.hudi.avro.model.HoodieCleanerPlan;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
@@ -110,6 +111,11 @@ public class AvroUtils {
return serializeAvroMetadata(compactionWorkload, HoodieCompactionPlan.class);
}
public static Option<byte[]> serializeCleanerPlan(HoodieCleanerPlan cleanPlan) throws IOException {
return serializeAvroMetadata(cleanPlan, HoodieCleanerPlan.class);
}
public static Option<byte[]> serializeCleanMetadata(HoodieCleanMetadata metadata) throws IOException {
return serializeAvroMetadata(metadata, HoodieCleanMetadata.class);
}
@@ -137,6 +143,10 @@ public class AvroUtils {
return Option.of(baos.toByteArray());
}
public static HoodieCleanerPlan deserializeCleanerPlan(byte[] bytes) throws IOException {
return deserializeAvroMetadata(bytes, HoodieCleanerPlan.class);
}
public static HoodieCompactionPlan deserializeCompactionPlan(byte[] bytes) throws IOException {
return deserializeAvroMetadata(bytes, HoodieCompactionPlan.class);
}

View File

@@ -141,10 +141,9 @@ public class ParquetUtils {
* Read out the bloom filter from the parquet file meta data.
*/
public static BloomFilter readBloomFilterFromParquetMetadata(Configuration configuration, Path parquetFilePath) {
Map<String, String> footerVals =
readParquetFooter(configuration, false, parquetFilePath,
HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY,
HoodieAvroWriteSupport.OLD_HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY);
Map<String, String> footerVals = readParquetFooter(configuration, false, parquetFilePath,
HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY,
HoodieAvroWriteSupport.OLD_HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY);
String footerVal = footerVals.get(HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY);
if (null == footerVal) {
// We use old style key "com.uber.hoodie.bloomfilter"

View File

@@ -27,6 +27,7 @@ import org.apache.hudi.common.util.collection.Pair;
/**
* Migrates a specific metadata type stored in .hoodie folder to latest version
*
* @param <T>
*/
public class MetadataMigrator<T> {
@@ -36,15 +37,16 @@ public class MetadataMigrator<T> {
private final Integer oldestVersion;
public MetadataMigrator(HoodieTableMetaClient metaClient, List<VersionMigrator<T>> migratorList) {
migrators = migratorList.stream().map(m ->
Pair.of(m.getManagedVersion(), m)).collect(Collectors.toMap(Pair::getKey, Pair::getValue));
migrators = migratorList.stream().map(m -> Pair.of(m.getManagedVersion(), m))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
latestVersion = migrators.keySet().stream().reduce((x, y) -> x > y ? x : y).get();
oldestVersion = migrators.keySet().stream().reduce((x, y) -> x < y ? x : y).get();
}
/**
* Upgrade Metadata version to its latest
* @param metadata Metadata
*
* @param metadata Metadata
* @param metadataVersion Current version of metadata
* @return Metadata conforming to the latest version of this metadata
*/
@@ -64,9 +66,10 @@ public class MetadataMigrator<T> {
/**
* Migrate metadata to a specific version
* @param metadata Hoodie Table Meta Client
* @param metadataVersion Metadata Version
* @param targetVersion Target Version
*
* @param metadata Hoodie Table Meta Client
* @param metadataVersion Metadata Version
* @param targetVersion Target Version
* @return Metadata conforming to the target version
*/
public T migrateToVersion(T metadata, int metadataVersion, int targetVersion) {

View File

@@ -22,18 +22,21 @@ import java.io.Serializable;
/**
* Responsible for upgrading and downgrading metadata versions for a specific metadata
*
* @param <T> Metadata Type
*/
public interface VersionMigrator<T> extends Serializable {
/**
* Version of Metadata that this class will handle
*
* @return
*/
Integer getManagedVersion();
/**
* Upgrades metadata of type T from previous version to this version
*
* @param input Metadata as of previous version.
* @return Metadata compatible with the version managed by this class
*/
@@ -41,6 +44,7 @@ public interface VersionMigrator<T> extends Serializable {
/**
* Downgrades metadata of type T from next version to this version
*
* @param input Metadata as of next highest version
* @return Metadata compatible with the version managed by this class
*/

View File

@@ -29,8 +29,7 @@ import org.apache.hudi.common.versioning.MetadataMigrator;
public class CompactionPlanMigrator extends MetadataMigrator<HoodieCompactionPlan> {
public CompactionPlanMigrator(HoodieTableMetaClient metaClient) {
super(metaClient, Arrays.asList(
new CompactionV1MigrationHandler(metaClient),
new CompactionV2MigrationHandler(metaClient)));
super(metaClient,
Arrays.asList(new CompactionV1MigrationHandler(metaClient), new CompactionV2MigrationHandler(metaClient)));
}
}

View File

@@ -52,21 +52,17 @@ public class CompactionV1MigrationHandler extends AbstractMigratorBase<HoodieCom
@Override
public HoodieCompactionPlan downgradeFrom(HoodieCompactionPlan input) {
Preconditions.checkArgument(input.getVersion() == 2, "Input version is "
+ input.getVersion() + ". Must be 2");
Preconditions.checkArgument(input.getVersion() == 2, "Input version is " + input.getVersion() + ". Must be 2");
HoodieCompactionPlan compactionPlan = new HoodieCompactionPlan();
final Path basePath = new Path(metaClient.getBasePath());
List<HoodieCompactionOperation> v1CompactionOperationList = new ArrayList<>();
if (null != input.getOperations()) {
v1CompactionOperationList = input.getOperations().stream().map(inp -> {
return HoodieCompactionOperation.newBuilder()
.setBaseInstantTime(inp.getBaseInstantTime())
.setFileId(inp.getFileId())
.setPartitionPath(inp.getPartitionPath())
.setMetrics(inp.getMetrics())
return HoodieCompactionOperation.newBuilder().setBaseInstantTime(inp.getBaseInstantTime())
.setFileId(inp.getFileId()).setPartitionPath(inp.getPartitionPath()).setMetrics(inp.getMetrics())
.setDataFilePath(convertToV1Path(basePath, inp.getPartitionPath(), inp.getDataFilePath()))
.setDeltaFilePaths(inp.getDeltaFilePaths().stream().map(s -> convertToV1Path(basePath,
inp.getPartitionPath(), s)).collect(Collectors.toList()))
.setDeltaFilePaths(inp.getDeltaFilePaths().stream()
.map(s -> convertToV1Path(basePath, inp.getPartitionPath(), s)).collect(Collectors.toList()))
.build();
}).collect(Collectors.toList());
}

View File

@@ -46,20 +46,15 @@ public class CompactionV2MigrationHandler extends AbstractMigratorBase<HoodieCom
@Override
public HoodieCompactionPlan upgradeFrom(HoodieCompactionPlan input) {
Preconditions.checkArgument(input.getVersion() == 1, "Input version is "
+ input.getVersion() + ". Must be 1");
Preconditions.checkArgument(input.getVersion() == 1, "Input version is " + input.getVersion() + ". Must be 1");
HoodieCompactionPlan compactionPlan = new HoodieCompactionPlan();
List<HoodieCompactionOperation> v2CompactionOperationList = new ArrayList<>();
if (null != input.getOperations()) {
v2CompactionOperationList = input.getOperations().stream().map(inp -> {
return HoodieCompactionOperation.newBuilder()
.setBaseInstantTime(inp.getBaseInstantTime())
.setFileId(inp.getFileId())
.setPartitionPath(inp.getPartitionPath())
.setMetrics(inp.getMetrics())
.setDataFilePath(new Path(inp.getDataFilePath()).getName())
.setDeltaFilePaths(inp.getDeltaFilePaths().stream().map(s -> new Path(s).getName())
.collect(Collectors.toList()))
return HoodieCompactionOperation.newBuilder().setBaseInstantTime(inp.getBaseInstantTime())
.setFileId(inp.getFileId()).setPartitionPath(inp.getPartitionPath()).setMetrics(inp.getMetrics())
.setDataFilePath(new Path(inp.getDataFilePath()).getName()).setDeltaFilePaths(
inp.getDeltaFilePaths().stream().map(s -> new Path(s).getName()).collect(Collectors.toList()))
.build();
}).collect(Collectors.toList());
}