prashantwason commented on a change in pull request #2496:
URL: https://github.com/apache/hudi/pull/2496#discussion_r567157247
##########
File path:
hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieWrapperFileSystem.java
##########
@@ -192,76 +233,110 @@ public FSDataOutputStream create(Path f, FsPermission
permission, boolean overwr
return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> {
final Path translatedPath = convertToDefaultPath(f);
return wrapOutputStream(f,
- fileSystem.create(translatedPath, permission, overwrite, bufferSize,
replication, blockSize, progress));
+ fileSystem.create(translatedPath, permission, overwrite, bufferSize,
replication, blockSize, progress), bufferSize);
});
}
- private FSDataOutputStream wrapOutputStream(final Path path,
FSDataOutputStream fsDataOutputStream)
+
+ /**
+ * The stream hierarchy after wrapping will be as follows.
+ *
+ * FSDataOuputStream (returned)
+ * BufferedOutputStream (required for output buffering)
+ * TimedSizeAwareOutputStream (required for tracking metrics,
timings and the number of bytes written)
+ * fs.open() (Original stream returned from underlying
FileSystem)
+ */
+ private FSDataOutputStream wrapOutputStream(final Path path,
FSDataOutputStream fsDataOutputStream, int bufferSize)
throws IOException {
- if (fsDataOutputStream instanceof SizeAwareFSDataOutputStream) {
- return fsDataOutputStream;
- }
+ int minBufferSize = (HoodieFileFormat.isBaseFile(path) ||
FSUtils.isLogFile(path)) ? MIN_DATA_FILE_IO_BUFFER_SIZE
+ : MIN_IO_BUFFER_SIZE;
+ int finalBufferSize = Math.max(minBufferSize, bufferSize);
+ LOG.info("Using buffersize=" + finalBufferSize + " to wrap output stream
for path " + path);
- SizeAwareFSDataOutputStream os = new SizeAwareFSDataOutputStream(path,
fsDataOutputStream, consistencyGuard,
+ TimedSizeAwareOutputStream tos = new TimedSizeAwareOutputStream(path,
fsDataOutputStream, consistencyGuard,
() -> openStreams.remove(path.getName()));
- openStreams.put(path.getName(), os);
- return os;
+ BufferedSizeAwareOutputStream bos = new BufferedSizeAwareOutputStream(tos,
finalBufferSize);
+
+ openStreams.put(path.getName(), Pair.of(tos, bos));
Review comment:
Yes, the outerstream.close() is closing the wrapped streams.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]