jonvex commented on code in PR #11048: URL: https://github.com/apache/hudi/pull/11048#discussion_r1569817603
########## hudi-io/src/main/java/org/apache/hudi/storage/HoodieStorage.java: ########## @@ -332,6 +350,18 @@ public boolean createNewFile(StoragePath path) throws IOException { } } + /** + * Opens an SeekableDataInputStream at the indicated path with seeks supported. + * + * @param path the file to open. + * @return the InputStream to read from. + * @throws IOException IO error. + */ + @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) + public SeekableDataInputStream openSeekable(StoragePath path) throws IOException { Review Comment: minor: should put this next to the other openSeekable def ########## hudi-io/src/test/java/org/apache/hudi/io/storage/TestHoodieStorageBase.java: ########## @@ -148,6 +150,46 @@ public void testCreateWriteAndRead() throws IOException { assertTrue(storage.createDirectory(path4)); } + @Test + public void testSeekable() throws IOException { + HoodieStorage storage = getHoodieStorage(); + StoragePath path = new StoragePath(getTempDir(), "testSeekable/1.file"); + assertFalse(storage.exists(path)); + byte[] data = new byte[] {2, 42, 49, (byte) 158, (byte) 233, 66, 9, 34, 79}; + + // By default, create overwrites the file + try (OutputStream stream = storage.create(path)) { + stream.write(data); + stream.flush(); + } + + try (SeekableDataInputStream seekableStream = storage.openSeekable(path)) { + validateSeekableDataInputStream(seekableStream, data); + } + + try (SeekableDataInputStream seekableStream = storage.openSeekable(path, 2)) { + validateSeekableDataInputStream(seekableStream, data); + } + } + + private void validateSeekableDataInputStream(SeekableDataInputStream seekableStream, + byte[] expectedData) throws IOException { + List<Integer> positionList = new ArrayList<>(); + positionList.add(1); + positionList.add(expectedData.length / 2); + positionList.add(expectedData.length - 1); Review Comment: why add these 3 if they will already be in the range [0,expectedData.length) ? ########## hudi-hadoop-common/src/main/java/org/apache/hudi/hadoop/fs/HadoopFSUtils.java: ########## @@ -154,4 +158,90 @@ public static FileStatus convertToHadoopFileStatus(StoragePathInfo pathInfo) { pathInfo.getModificationTime(), convertToHadoopPath(pathInfo.getPath())); } + + /** + * Fetch the right {@link FSDataInputStream} to be used by wrapping with required input streams. + * + * @param fs instance of {@link FileSystem} in use. + * @param filePath path of the file. + * @param bufferSize buffer size to be used. + * @return the right {@link FSDataInputStream} as required. + */ + public static FSDataInputStream getFSDataInputStream(FileSystem fs, + StoragePath filePath, + int bufferSize) { + FSDataInputStream fsDataInputStream = null; + try { + fsDataInputStream = fs.open(new Path(filePath.toUri()), bufferSize); Review Comment: What do you think about a helper method for this?: new Path(filePath.toUri()) or will that just add more clutter? ########## hudi-io/src/test/java/org/apache/hudi/io/storage/TestHoodieStorageBase.java: ########## @@ -148,6 +150,46 @@ public void testCreateWriteAndRead() throws IOException { assertTrue(storage.createDirectory(path4)); } + @Test + public void testSeekable() throws IOException { + HoodieStorage storage = getHoodieStorage(); + StoragePath path = new StoragePath(getTempDir(), "testSeekable/1.file"); + assertFalse(storage.exists(path)); + byte[] data = new byte[] {2, 42, 49, (byte) 158, (byte) 233, 66, 9, 34, 79}; Review Comment: why did you need to cast the values over 100? -- 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. To unsubscribe, e-mail: commits-unsubscr...@hudi.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org