vinothchandar commented on a change in pull request #2496:
URL: https://github.com/apache/hudi/pull/2496#discussion_r567056507



##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieWrapperFileSystem.java
##########
@@ -79,8 +82,16 @@ public static void setMetricsRegistry(Registry registry, 
Registry registryMeta)
     METRICS_REGISTRY_META = registryMeta;
   }
 
+  // Minimum buffer size of data files and log files which are generally large 
in size
+  public static final int MIN_DATA_FILE_IO_BUFFER_SIZE = 16 * 1024 * 1024; // 
16 MB

Review comment:
       rename: `MIN_HOODIE_FILE_IO_BUFFER_SIZE` :) 

##########
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:
       do we need to close both here? typically, the outerStream.close() should 
also closer the inner/wrapped streams right

##########
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));
+
+    return new FSDataOutputStream(bos);
   }
 
-  private FSDataInputStream wrapInputStream(final Path path, FSDataInputStream 
fsDataInputStream) throws IOException {
-    if (fsDataInputStream instanceof TimedFSDataInputStream) {
-      return fsDataInputStream;
-    }
-    return new TimedFSDataInputStream(path, fsDataInputStream);
+  /**
+   * The stream hierarchy after wrapping will be as follows.
+   *
+   * FSDataInputStream (returned)
+   *    BufferedFSInputStream (required for buffering)
+   *      TimedFSInputStream  (required for tracking metrics and timings)
+   *         fs.open()  (Original stream returned from underlying FileSystem)
+   *
+   * @param path
+   * @param fsDataInputStream
+   * @param bufferSize
+   * @return
+   * @throws IOException
+   */
+
+  private FSDataInputStream wrapInputStream(final Path path, FSDataInputStream 
fsDataInputStream,
+      int bufferSize) throws IOException {
+    LOG.info("Using buffersize=" + bufferSize + " to wrap input stream for 
path " + path);
+    TimedFSInputStream tis = new TimedFSInputStream(path, fsDataInputStream);
+
+    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);
+    return new FSDataInputStream(new BufferedFSInputStream(tis, 
finalBufferSize));
   }
 
   @Override
   public FSDataOutputStream create(Path f, boolean overwrite) throws 
IOException {
     return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> {
-      return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), 
overwrite));
+      return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), 
overwrite), 0);
     });
   }
 
   @Override
   public FSDataOutputStream create(Path f) throws IOException {
     return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> {
-      return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f)));
+      return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f)), 
0);
     });
   }
 
   @Override
   public FSDataOutputStream create(Path f, Progressable progress) throws 
IOException {
     return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> {
-      return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), 
progress));
+      return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), 
progress), 0);

Review comment:
       create a overloaded method for `wrapOutputStream(..)` that passes `0` 
for the last argument?

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieWrapperFileSystem.java
##########
@@ -79,8 +82,16 @@ public static void setMetricsRegistry(Registry registry, 
Registry registryMeta)
     METRICS_REGISTRY_META = registryMeta;
   }
 
+  // Minimum buffer size of data files and log files which are generally large 
in size
+  public static final int MIN_DATA_FILE_IO_BUFFER_SIZE = 16 * 1024 * 1024; // 
16 MB

Review comment:
       I think this is small enough to not a large overhead even if S3/GCS are 
already buffering. 




----------------------------------------------------------------
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]


Reply via email to