1
0

[HUDI-1434] fix incorrect log file path in HoodieWriteStat (#2300)

* [HUDI-1434] fix incorrect log file path in HoodieWriteStat

* HoodieWriteHandle#close() returns a list of WriteStatus objs

* Handle rolled-over log files and return a WriteStatus per log file written

 - Combined data and delete block logging into a single call
 - Lazily initialize and manage write status based on returned AppendResult
 - Use FSUtils.getFileSize() to set final file size, consistent with other handles
 - Added tests around returned values in AppendResult
 - Added validation of the file sizes returned in write stat

Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
Gary Li
2020-12-31 06:22:15 +08:00
committed by GitHub
parent ef28763f08
commit 605b617cfa
29 changed files with 591 additions and 298 deletions

View File

@@ -20,6 +20,9 @@ package org.apache.hudi.common.model;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import java.util.ArrayList;
import java.util.List;
/**
* Statistics about a single Hoodie delta log operation.
*/
@@ -28,6 +31,8 @@ public class HoodieDeltaWriteStat extends HoodieWriteStat {
private int logVersion;
private long logOffset;
private String baseFile;
private List<String> logFiles = new ArrayList<>();
public void setLogVersion(int logVersion) {
this.logVersion = logVersion;
@@ -44,4 +49,24 @@ public class HoodieDeltaWriteStat extends HoodieWriteStat {
public long getLogOffset() {
return logOffset;
}
public void setBaseFile(String baseFile) {
this.baseFile = baseFile;
}
public String getBaseFile() {
return baseFile;
}
public void setLogFiles(List<String> logFiles) {
this.logFiles = logFiles;
}
public void addLogFiles(String logFile) {
logFiles.add(logFile);
}
public List<String> getLogFiles() {
return logFiles;
}
}

View File

@@ -63,6 +63,12 @@ public class HoodieLogFile implements Serializable {
this.fileLen = 0;
}
public HoodieLogFile(Path logPath, Long fileLen) {
this.fileStatus = null;
this.pathStr = logPath.toString();
this.fileLen = fileLen;
}
public HoodieLogFile(String logPathStr) {
this.fileStatus = null;
this.pathStr = logPathStr;

View File

@@ -71,7 +71,7 @@ public class HoodieWriteStat implements Serializable {
private long numInserts;
/**
* Total size of file written.
* Total number of bytes written.
*/
private long totalWriteBytes;

View File

@@ -0,0 +1,50 @@
/*
* 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.
*/
package org.apache.hudi.common.table.log;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
/**
* Pojo holding information on the result of a {@link org.apache.hudi.common.table.log.HoodieLogFormat.Writer#appendBlock(HoodieLogBlock)}.
*/
public class AppendResult {
private final HoodieLogFile logFile;
private final long offset;
private final long size;
public AppendResult(HoodieLogFile logFile, long offset, long size) {
this.logFile = logFile;
this.offset = offset;
this.size = size;
}
public HoodieLogFile logFile() {
return logFile;
}
public long offset() {
return offset;
}
public long size() {
return size;
}
}

View File

@@ -33,6 +33,7 @@ import org.apache.log4j.Logger;
import java.io.Closeable;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
/**
* File Format for Hoodie Log Files. The File Format consists of blocks each separated with a MAGIC sync marker. A Block
@@ -61,14 +62,21 @@ public interface HoodieLogFormat {
interface Writer extends Closeable {
/**
* @return the path to this {@link HoodieLogFormat}
* @return the path to the current {@link HoodieLogFile} being written to.
*/
HoodieLogFile getLogFile();
/**
* Append Block returns a new Writer if the log is rolled.
* Append Block to a log file.
* @return {@link AppendResult} containing result of the append.
*/
Writer appendBlock(HoodieLogBlock block) throws IOException, InterruptedException;
AppendResult appendBlock(HoodieLogBlock block) throws IOException, InterruptedException;
/**
* Appends the list of blocks to a logfile.
* @return {@link AppendResult} containing result of the append.
*/
AppendResult appendBlocks(List<HoodieLogBlock> blocks) throws IOException, InterruptedException;
long getCurrentSize() throws IOException;
}
@@ -88,7 +96,7 @@ public interface HoodieLogFormat {
*
* @return
*/
public boolean hasPrev();
boolean hasPrev();
/**
* Read log file in reverse order and return prev block if present.
@@ -96,7 +104,7 @@ public interface HoodieLogFormat {
* @return
* @throws IOException
*/
public HoodieLogBlock prev() throws IOException;
HoodieLogBlock prev() throws IOException;
}
/**
@@ -125,6 +133,8 @@ public interface HoodieLogFormat {
// version number for this log file. If not specified, then the current version will be
// computed by inspecting the file system
private Integer logVersion;
// file len of this log file
private Long fileLen = 0L;
// Location of the directory containing the log
private Path parentPath;
// Log File Write Token
@@ -142,13 +152,13 @@ public interface HoodieLogFormat {
return this;
}
public WriterBuilder withLogWriteToken(String writeToken) {
this.logWriteToken = writeToken;
public WriterBuilder withRolloverLogWriteToken(String rolloverLogWriteToken) {
this.rolloverLogWriteToken = rolloverLogWriteToken;
return this;
}
public WriterBuilder withRolloverLogWriteToken(String rolloverLogWriteToken) {
this.rolloverLogWriteToken = rolloverLogWriteToken;
public WriterBuilder withLogWriteToken(String logWriteToken) {
this.logWriteToken = logWriteToken;
return this;
}
@@ -182,12 +192,17 @@ public interface HoodieLogFormat {
return this;
}
public WriterBuilder withFileSize(long fileLen) {
this.fileLen = fileLen;
return this;
}
public WriterBuilder onParentPath(Path parentPath) {
this.parentPath = parentPath;
return this;
}
public Writer build() throws IOException, InterruptedException {
public Writer build() throws IOException {
LOG.info("Building HoodieLogFormat Writer");
if (fs == null) {
throw new IllegalArgumentException("fs is not specified");
@@ -229,13 +244,14 @@ public interface HoodieLogFormat {
if (logWriteToken == null) {
// This is the case where we have existing log-file with old format. rollover to avoid any conflicts
logVersion += 1;
fileLen = 0L;
logWriteToken = rolloverLogWriteToken;
}
Path logPath = new Path(parentPath,
FSUtils.makeLogFileName(logFileId, fileExtension, instantTime, logVersion, logWriteToken));
LOG.info("HoodieLogFile on path " + logPath);
HoodieLogFile logFile = new HoodieLogFile(logPath);
HoodieLogFile logFile = new HoodieLogFile(logPath, fileLen);
if (bufferSize == null) {
bufferSize = FSUtils.getDefaultBufferSize(fs);
@@ -246,8 +262,7 @@ public interface HoodieLogFormat {
if (sizeThreshold == null) {
sizeThreshold = DEFAULT_SIZE_THRESHOLD;
}
return new HoodieLogFormatWriter(fs, logFile, bufferSize, replication, sizeThreshold, logWriteToken,
rolloverLogWriteToken);
return new HoodieLogFormatWriter(fs, logFile, bufferSize, replication, sizeThreshold, rolloverLogWriteToken);
}
}

View File

@@ -21,7 +21,6 @@ package org.apache.hudi.common.table.log;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.fs.StorageSchemes;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
import org.apache.hudi.common.table.log.HoodieLogFormat.WriterBuilder;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.exception.HoodieException;
@@ -38,6 +37,8 @@ import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
/**
* HoodieLogFormatWriter can be used to append blocks to a log file Use HoodieLogFormat.WriterBuilder to construct.
@@ -47,13 +48,13 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
private static final Logger LOG = LogManager.getLogger(HoodieLogFormatWriter.class);
private HoodieLogFile logFile;
private FSDataOutputStream output;
private final FileSystem fs;
private final long sizeThreshold;
private final Integer bufferSize;
private final Short replication;
private final String logWriteToken;
private final String rolloverLogWriteToken;
private FSDataOutputStream output;
private boolean closed = false;
private transient Thread shutdownThread = null;
@@ -66,14 +67,12 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
* @param replication
* @param sizeThreshold
*/
HoodieLogFormatWriter(FileSystem fs, HoodieLogFile logFile, Integer bufferSize, Short replication, Long sizeThreshold,
String logWriteToken, String rolloverLogWriteToken) {
HoodieLogFormatWriter(FileSystem fs, HoodieLogFile logFile, Integer bufferSize, Short replication, Long sizeThreshold, String rolloverLogWriteToken) {
this.fs = fs;
this.logFile = logFile;
this.sizeThreshold = sizeThreshold;
this.bufferSize = bufferSize;
this.replication = replication;
this.logWriteToken = logWriteToken;
this.rolloverLogWriteToken = rolloverLogWriteToken;
addShutDownHook();
}
@@ -105,6 +104,7 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
if (isAppendSupported) {
LOG.info(logFile + " exists. Appending to existing file");
try {
// open the path for append and record the offset
this.output = fs.append(path, bufferSize);
} catch (RemoteException e) {
LOG.warn("Remote Exception, attempting to handle or recover lease", e);
@@ -124,9 +124,9 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
}
}
if (!isAppendSupported) {
this.logFile = logFile.rollOver(fs, rolloverLogWriteToken);
LOG.info("Append not supported.. Rolling over to " + logFile);
rollOver();
createNewFile();
LOG.info("Append not supported.. Rolling over to " + logFile);
}
} else {
LOG.info(logFile + " does not exist. Create a new file");
@@ -138,52 +138,66 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
}
@Override
public Writer appendBlock(HoodieLogBlock block) throws IOException, InterruptedException {
public AppendResult appendBlock(HoodieLogBlock block) throws IOException, InterruptedException {
return appendBlocks(Collections.singletonList(block));
}
@Override
public AppendResult appendBlocks(List<HoodieLogBlock> blocks) throws IOException, InterruptedException {
// Find current version
HoodieLogFormat.LogFormatVersion currentLogFormatVersion =
new HoodieLogFormatVersion(HoodieLogFormat.CURRENT_VERSION);
FSDataOutputStream outputStream = getOutputStream();
long currentSize = outputStream.size();
long startPos = outputStream.getPos();
long sizeWritten = 0;
// 1. Write the magic header for the start of the block
outputStream.write(HoodieLogFormat.MAGIC);
for (HoodieLogBlock block: blocks) {
long startSize = outputStream.size();
// bytes for header
byte[] headerBytes = HoodieLogBlock.getLogMetadataBytes(block.getLogBlockHeader());
// content bytes
byte[] content = block.getContentBytes();
// bytes for footer
byte[] footerBytes = HoodieLogBlock.getLogMetadataBytes(block.getLogBlockFooter());
// 1. Write the magic header for the start of the block
outputStream.write(HoodieLogFormat.MAGIC);
// 2. Write the total size of the block (excluding Magic)
outputStream.writeLong(getLogBlockLength(content.length, headerBytes.length, footerBytes.length));
// bytes for header
byte[] headerBytes = HoodieLogBlock.getLogMetadataBytes(block.getLogBlockHeader());
// content bytes
byte[] content = block.getContentBytes();
// bytes for footer
byte[] footerBytes = HoodieLogBlock.getLogMetadataBytes(block.getLogBlockFooter());
// 3. Write the version of this log block
outputStream.writeInt(currentLogFormatVersion.getVersion());
// 4. Write the block type
outputStream.writeInt(block.getBlockType().ordinal());
// 2. Write the total size of the block (excluding Magic)
outputStream.writeLong(getLogBlockLength(content.length, headerBytes.length, footerBytes.length));
// 5. Write the headers for the log block
outputStream.write(headerBytes);
// 6. Write the size of the content block
outputStream.writeLong(content.length);
// 7. Write the contents of the data block
outputStream.write(content);
// 8. Write the footers for the log block
outputStream.write(footerBytes);
// 9. Write the total size of the log block (including magic) which is everything written
// until now (for reverse pointer)
// Update: this information is now used in determining if a block is corrupt by comparing to the
// block size in header. This change assumes that the block size will be the last data written
// to a block. Read will break if any data is written past this point for a block.
outputStream.writeLong(outputStream.size() - currentSize);
// Flush every block to disk
// 3. Write the version of this log block
outputStream.writeInt(currentLogFormatVersion.getVersion());
// 4. Write the block type
outputStream.writeInt(block.getBlockType().ordinal());
// 5. Write the headers for the log block
outputStream.write(headerBytes);
// 6. Write the size of the content block
outputStream.writeLong(content.length);
// 7. Write the contents of the data block
outputStream.write(content);
// 8. Write the footers for the log block
outputStream.write(footerBytes);
// 9. Write the total size of the log block (including magic) which is everything written
// until now (for reverse pointer)
// Update: this information is now used in determining if a block is corrupt by comparing to the
// block size in header. This change assumes that the block size will be the last data written
// to a block. Read will break if any data is written past this point for a block.
outputStream.writeLong(outputStream.size() - startSize);
// Fetch the size again, so it accounts also (9).
sizeWritten += outputStream.size() - startSize;
}
// Flush all blocks to disk
flush();
AppendResult result = new AppendResult(logFile, startPos, sizeWritten);
// roll over if size is past the threshold
return rolloverIfNeeded();
rolloverIfNeeded();
return result;
}
/**
@@ -201,20 +215,19 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
Long.BYTES; // bytes to write totalLogBlockLength at end of block (for reverse ptr)
}
private Writer rolloverIfNeeded() throws IOException, InterruptedException {
private void rolloverIfNeeded() throws IOException {
// Roll over if the size is past the threshold
if (getCurrentSize() > sizeThreshold) {
// TODO - make an end marker which seals the old log file (no more appends possible to that
// file).
LOG.info("CurrentSize " + getCurrentSize() + " has reached threshold " + sizeThreshold
+ ". Rolling over to the next version");
HoodieLogFile newLogFile = logFile.rollOver(fs, rolloverLogWriteToken);
// close this writer and return the new writer
close();
return new HoodieLogFormatWriter(fs, newLogFile, bufferSize, replication, sizeThreshold, logWriteToken,
rolloverLogWriteToken);
rollOver();
}
return this;
}
private void rollOver() throws IOException {
closeStream();
this.logFile = logFile.rollOver(fs, rolloverLogWriteToken);
this.closed = false;
}
private void createNewFile() throws IOException {
@@ -292,13 +305,12 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
// appended to, then the NN will throw an exception saying that it couldn't find any active replica with the
// last block. Find more information here : https://issues.apache.org/jira/browse/HDFS-6325
LOG.warn("Failed to open an append stream to the log file. Opening a new log file..", e);
// Rollover the current log file (since cannot get a stream handle) and create new one
this.logFile = logFile.rollOver(fs, rolloverLogWriteToken);
rollOver();
createNewFile();
} else if (e.getClassName().contentEquals(AlreadyBeingCreatedException.class.getName())) {
LOG.warn("Another task executor writing to the same log file(" + logFile + ". Rolling over");
// Rollover the current log file (since cannot get a stream handle) and create new one
this.logFile = logFile.rollOver(fs, rolloverLogWriteToken);
rollOver();
createNewFile();
} else if (e.getClassName().contentEquals(RecoveryInProgressException.class.getName())
&& (fs instanceof DistributedFileSystem)) {
@@ -311,8 +323,9 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
// try again
this.output = fs.append(path, bufferSize);
} else {
LOG.warn("Failed to recover lease on path " + path);
throw new HoodieException(e);
final String msg = "Failed to recover lease on path " + path;
LOG.warn(msg);
throw new HoodieException(msg, e);
}
} else {
// When fs.append() has failed and an exception is thrown, by closing the output stream
@@ -320,16 +333,16 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
// new attemptId, say taskId.1) it will be able to acquire lease on the log file (as output stream was
// closed properly by taskId.0).
//
// If close() call were to fail throwing an exception, our best bet is to rollover to a new log file.
// If closeStream() call were to fail throwing an exception, our best bet is to rollover to a new log file.
try {
close();
closeStream();
// output stream has been successfully closed and lease on the log file has been released,
// before throwing an exception for the append failure.
throw new HoodieIOException("Failed to append to the output stream ", e);
} catch (Exception ce) {
LOG.warn("Failed to close the output stream for " + fs.getClass().getName() + " on path " + path
+ ". Rolling over to a new log file.");
this.logFile = logFile.rollOver(fs, rolloverLogWriteToken);
rollOver();
createNewFile();
}
}