[HUDI-840] Avoid blank file created by HoodieLogFormatWriter (#1567)
This commit is contained in:
@@ -54,6 +54,7 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
|
||||
private final String logWriteToken;
|
||||
private final String rolloverLogWriteToken;
|
||||
private FSDataOutputStream output;
|
||||
private boolean closed = false;
|
||||
private static final String APPEND_UNAVAILABLE_EXCEPTION_MESSAGE = "not sufficiently replicated yet";
|
||||
|
||||
/**
|
||||
@@ -64,7 +65,7 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
|
||||
* @param sizeThreshold
|
||||
*/
|
||||
HoodieLogFormatWriter(FileSystem fs, HoodieLogFile logFile, Integer bufferSize, Short replication, Long sizeThreshold,
|
||||
String logWriteToken, String rolloverLogWriteToken) throws IOException, InterruptedException {
|
||||
String logWriteToken, String rolloverLogWriteToken) {
|
||||
this.fs = fs;
|
||||
this.logFile = logFile;
|
||||
this.sizeThreshold = sizeThreshold;
|
||||
@@ -73,40 +74,6 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
|
||||
this.logWriteToken = logWriteToken;
|
||||
this.rolloverLogWriteToken = rolloverLogWriteToken;
|
||||
addShutDownHook();
|
||||
Path path = logFile.getPath();
|
||||
if (fs.exists(path)) {
|
||||
boolean isAppendSupported = StorageSchemes.isAppendSupported(fs.getScheme());
|
||||
if (isAppendSupported) {
|
||||
LOG.info(logFile + " exists. Appending to existing file");
|
||||
try {
|
||||
this.output = fs.append(path, bufferSize);
|
||||
} catch (RemoteException e) {
|
||||
LOG.warn("Remote Exception, attempting to handle or recover lease", e);
|
||||
handleAppendExceptionOrRecoverLease(path, e);
|
||||
} catch (IOException ioe) {
|
||||
if (ioe.getMessage().toLowerCase().contains("not supported")) {
|
||||
// may still happen if scheme is viewfs.
|
||||
isAppendSupported = false;
|
||||
} else {
|
||||
/*
|
||||
* Before throwing an exception, close the outputstream,
|
||||
* to ensure that the lease on the log file is released.
|
||||
*/
|
||||
close();
|
||||
throw ioe;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (!isAppendSupported) {
|
||||
this.logFile = logFile.rollOver(fs, rolloverLogWriteToken);
|
||||
LOG.info("Append not supported.. Rolling over to " + logFile);
|
||||
createNewFile();
|
||||
}
|
||||
} else {
|
||||
LOG.info(logFile + " does not exist. Create a new file");
|
||||
// Block size does not matter as we will always manually autoflush
|
||||
createNewFile();
|
||||
}
|
||||
}
|
||||
|
||||
public FileSystem getFs() {
|
||||
@@ -122,16 +89,64 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
|
||||
return sizeThreshold;
|
||||
}
|
||||
|
||||
/**
|
||||
* Lazily opens the output stream if needed for writing.
|
||||
* @return OutputStream for writing to current log file.
|
||||
* @throws IOException
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
private FSDataOutputStream getOutputStream() throws IOException, InterruptedException {
|
||||
if (this.output == null) {
|
||||
Path path = logFile.getPath();
|
||||
if (fs.exists(path)) {
|
||||
boolean isAppendSupported = StorageSchemes.isAppendSupported(fs.getScheme());
|
||||
if (isAppendSupported) {
|
||||
LOG.info(logFile + " exists. Appending to existing file");
|
||||
try {
|
||||
this.output = fs.append(path, bufferSize);
|
||||
} catch (RemoteException e) {
|
||||
LOG.warn("Remote Exception, attempting to handle or recover lease", e);
|
||||
handleAppendExceptionOrRecoverLease(path, e);
|
||||
} catch (IOException ioe) {
|
||||
if (ioe.getMessage().toLowerCase().contains("not supported")) {
|
||||
// may still happen if scheme is viewfs.
|
||||
isAppendSupported = false;
|
||||
} else {
|
||||
/*
|
||||
* Before throwing an exception, close the outputstream,
|
||||
* to ensure that the lease on the log file is released.
|
||||
*/
|
||||
close();
|
||||
throw ioe;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (!isAppendSupported) {
|
||||
this.logFile = logFile.rollOver(fs, rolloverLogWriteToken);
|
||||
LOG.info("Append not supported.. Rolling over to " + logFile);
|
||||
createNewFile();
|
||||
}
|
||||
} else {
|
||||
LOG.info(logFile + " does not exist. Create a new file");
|
||||
// Block size does not matter as we will always manually autoflush
|
||||
createNewFile();
|
||||
}
|
||||
}
|
||||
return output;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Writer appendBlock(HoodieLogBlock block) throws IOException, InterruptedException {
|
||||
|
||||
// Find current version
|
||||
HoodieLogFormat.LogFormatVersion currentLogFormatVersion =
|
||||
new HoodieLogFormatVersion(HoodieLogFormat.CURRENT_VERSION);
|
||||
long currentSize = this.output.size();
|
||||
|
||||
FSDataOutputStream outputStream = getOutputStream();
|
||||
long currentSize = outputStream.size();
|
||||
|
||||
// 1. Write the magic header for the start of the block
|
||||
this.output.write(HoodieLogFormat.MAGIC);
|
||||
outputStream.write(HoodieLogFormat.MAGIC);
|
||||
|
||||
// bytes for header
|
||||
byte[] headerBytes = HoodieLogBlock.getLogMetadataBytes(block.getLogBlockHeader());
|
||||
@@ -141,27 +156,27 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
|
||||
byte[] footerBytes = HoodieLogBlock.getLogMetadataBytes(block.getLogBlockFooter());
|
||||
|
||||
// 2. Write the total size of the block (excluding Magic)
|
||||
this.output.writeLong(getLogBlockLength(content.length, headerBytes.length, footerBytes.length));
|
||||
outputStream.writeLong(getLogBlockLength(content.length, headerBytes.length, footerBytes.length));
|
||||
|
||||
// 3. Write the version of this log block
|
||||
this.output.writeInt(currentLogFormatVersion.getVersion());
|
||||
outputStream.writeInt(currentLogFormatVersion.getVersion());
|
||||
// 4. Write the block type
|
||||
this.output.writeInt(block.getBlockType().ordinal());
|
||||
outputStream.writeInt(block.getBlockType().ordinal());
|
||||
|
||||
// 5. Write the headers for the log block
|
||||
this.output.write(headerBytes);
|
||||
outputStream.write(headerBytes);
|
||||
// 6. Write the size of the content block
|
||||
this.output.writeLong(content.length);
|
||||
outputStream.writeLong(content.length);
|
||||
// 7. Write the contents of the data block
|
||||
this.output.write(content);
|
||||
outputStream.write(content);
|
||||
// 8. Write the footers for the log block
|
||||
this.output.write(footerBytes);
|
||||
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.
|
||||
this.output.writeLong(this.output.size() - currentSize);
|
||||
outputStream.writeLong(outputStream.size() - currentSize);
|
||||
// Flush every block to disk
|
||||
flush();
|
||||
|
||||
@@ -207,9 +222,12 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
flush();
|
||||
output.close();
|
||||
output = null;
|
||||
if (output != null) {
|
||||
flush();
|
||||
output.close();
|
||||
output = null;
|
||||
closed = true;
|
||||
}
|
||||
}
|
||||
|
||||
private void flush() throws IOException {
|
||||
@@ -224,9 +242,13 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
|
||||
|
||||
@Override
|
||||
public long getCurrentSize() throws IOException {
|
||||
if (output == null) {
|
||||
if (closed) {
|
||||
throw new IllegalStateException("Cannot get current size as the underlying stream has been closed already");
|
||||
}
|
||||
|
||||
if (output == null) {
|
||||
return 0;
|
||||
}
|
||||
return output.getPos();
|
||||
}
|
||||
|
||||
@@ -302,5 +324,4 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user