1
0

[HUDI-2833][Design] Merge small archive files instead of expanding indefinitely. (#4078)

Co-authored-by: yuezhang <yuezhang@freewheel.tv>
This commit is contained in:
YueZhang
2022-01-19 14:42:35 +08:00
committed by GitHub
parent 4bea758738
commit 7647562dad
8 changed files with 639 additions and 46 deletions

View File

@@ -0,0 +1,42 @@
/*
* 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":"HoodieMergeArchiveFilePlan",
"fields":[
{
"name":"version",
"type":["int", "null"],
"default": 1
},
{
"name":"candidate",
"type":["null", {
"type":"array",
"items": "string"
}],
"default": null
},
{
"name":"mergedArchiveFileName",
"type":["null", "string"],
"default": null
}
]
}

View File

@@ -361,7 +361,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
private void createFileInAuxiliaryFolder(HoodieInstant instant, Option<byte[]> data) {
// This will be removed in future release. See HUDI-546
Path fullPath = new Path(metaClient.getMetaAuxiliaryPath(), instant.getFileName());
createFileInPath(fullPath, data);
FileIOUtils.createFileInPath(metaClient.getFs(), fullPath, data);
}
//-----------------------------------------------------------------
@@ -505,7 +505,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
fromInstant.getFileName())));
// Use Write Once to create Target File
if (allowRedundantTransitions) {
createFileInPath(new Path(metaClient.getMetaPath(), toInstant.getFileName()), data);
FileIOUtils.createFileInPath(metaClient.getFs(), new Path(metaClient.getMetaPath(), toInstant.getFileName()), data);
} else {
createImmutableFileInPath(new Path(metaClient.getMetaPath(), toInstant.getFileName()), data);
}
@@ -602,33 +602,12 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
private void createFileInMetaPath(String filename, Option<byte[]> content, boolean allowOverwrite) {
Path fullPath = new Path(metaClient.getMetaPath(), filename);
if (allowOverwrite || metaClient.getTimelineLayoutVersion().isNullVersion()) {
createFileInPath(fullPath, content);
FileIOUtils.createFileInPath(metaClient.getFs(), fullPath, content);
} else {
createImmutableFileInPath(fullPath, content);
}
}
private void createFileInPath(Path fullPath, Option<byte[]> content) {
try {
// If the path does not exist, create it first
if (!metaClient.getFs().exists(fullPath)) {
if (metaClient.getFs().createNewFile(fullPath)) {
LOG.info("Created a new file in meta path: " + fullPath);
} else {
throw new HoodieIOException("Failed to create file " + fullPath);
}
}
if (content.isPresent()) {
FSDataOutputStream fsout = metaClient.getFs().create(fullPath, true);
fsout.write(content.get());
fsout.close();
}
} catch (IOException e) {
throw new HoodieIOException("Failed to create file " + fullPath, e);
}
}
/**
* Creates a new file in timeline with overwrite set to false. This ensures
* files are created only once and never rewritten

View File

@@ -20,13 +20,17 @@ package org.apache.hudi.common.table.timeline;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieArchivedMetaEntry;
import org.apache.hudi.avro.model.HoodieMergeArchiveFilePlan;
import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.util.CollectionUtils;
import org.apache.hudi.common.util.FileIOUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.avro.generic.GenericRecord;
@@ -46,6 +50,7 @@ import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -66,6 +71,7 @@ import java.util.stream.Collectors;
* This class can be serialized and de-serialized and on de-serialization the FileSystem is re-initialized.
*/
public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
public static final String MERGE_ARCHIVE_PLAN_NAME = "mergeArchivePlan";
private static final Pattern ARCHIVE_FILE_PATTERN =
Pattern.compile("^\\.commits_\\.archive\\.([0-9]+).*");
@@ -218,7 +224,7 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
// Sort files by version suffix in reverse (implies reverse chronological order)
Arrays.sort(fsStatuses, new ArchiveFileVersionComparator());
List<HoodieInstant> instantsInRange = new ArrayList<>();
Set<HoodieInstant> instantsInRange = new HashSet<>();
for (FileStatus fs : fsStatuses) {
// Read the archived file
try (HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(metaClient.getFs(),
@@ -248,11 +254,32 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
break;
}
}
} catch (Exception originalException) {
// merge small archive files may left uncompleted archive file which will cause exception.
// need to ignore this kind of exception here.
try {
Path planPath = new Path(metaClient.getArchivePath(), MERGE_ARCHIVE_PLAN_NAME);
HoodieWrapperFileSystem fileSystem = metaClient.getFs();
if (fileSystem.exists(planPath)) {
HoodieMergeArchiveFilePlan plan = TimelineMetadataUtils.deserializeAvroMetadata(FileIOUtils.readDataFromPath(fileSystem, planPath).get(), HoodieMergeArchiveFilePlan.class);
String mergedArchiveFileName = plan.getMergedArchiveFileName();
if (!StringUtils.isNullOrEmpty(mergedArchiveFileName) && fs.getPath().getName().equalsIgnoreCase(mergedArchiveFileName)) {
LOG.warn("Catch exception because of reading uncompleted merging archive file " + mergedArchiveFileName + ". Ignore it here.");
continue;
}
}
throw originalException;
} catch (Exception e) {
// If anything wrong during parsing merge archive plan, we need to throw the original exception.
// For example corrupted archive file and corrupted plan are both existed.
throw originalException;
}
}
}
Collections.sort(instantsInRange);
return instantsInRange;
ArrayList<HoodieInstant> result = new ArrayList<>(instantsInRange);
Collections.sort(result);
return result;
} catch (IOException e) {
throw new HoodieIOException(
"Could not load archived commit timeline from path " + metaClient.getArchivePath(), e);

View File

@@ -160,4 +160,48 @@ public class FileIOUtils {
LOG.warn("IOException during close", e);
}
}
public static void createFileInPath(FileSystem fileSystem, org.apache.hadoop.fs.Path fullPath, Option<byte[]> content, boolean ignoreIOE) {
try {
// If the path does not exist, create it first
if (!fileSystem.exists(fullPath)) {
if (fileSystem.createNewFile(fullPath)) {
LOG.info("Created a new file in meta path: " + fullPath);
} else {
throw new HoodieIOException("Failed to create file " + fullPath);
}
}
if (content.isPresent()) {
FSDataOutputStream fsout = fileSystem.create(fullPath, true);
fsout.write(content.get());
fsout.close();
}
} catch (IOException e) {
LOG.warn("Failed to create file " + fullPath, e);
if (!ignoreIOE) {
throw new HoodieIOException("Failed to create file " + fullPath, e);
}
}
}
public static void createFileInPath(FileSystem fileSystem, org.apache.hadoop.fs.Path fullPath, Option<byte[]> content) {
createFileInPath(fileSystem, fullPath, content, false);
}
public static Option<byte[]> readDataFromPath(FileSystem fileSystem, org.apache.hadoop.fs.Path detailPath, boolean ignoreIOE) {
try (FSDataInputStream is = fileSystem.open(detailPath)) {
return Option.of(FileIOUtils.readAsByteArray(is));
} catch (IOException e) {
LOG.warn("Could not read commit details from " + detailPath, e);
if (!ignoreIOE) {
throw new HoodieIOException("Could not read commit details from " + detailPath, e);
}
return Option.empty();
}
}
public static Option<byte[]> readDataFromPath(FileSystem fileSystem, org.apache.hadoop.fs.Path detailPath) {
return readDataFromPath(fileSystem, detailPath, false);
}
}