vburenin commented on a change in pull request #2500:
URL: https://github.com/apache/hudi/pull/2500#discussion_r575332449



##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/fs/SchemeAwareFSDataInputStream.java
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.fs;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * Scheme aware FSDataInputStream so that we manipulate seeks for GS 
filesystem.
+ */
+public class SchemeAwareFSDataInputStream extends FSDataInputStream {
+
+  private final boolean isGCSFileSystem;
+
+  public SchemeAwareFSDataInputStream(InputStream in, boolean isGCSFileSystem) 
{
+    super(in);
+    this.isGCSFileSystem = isGCSFileSystem;
+  }
+
+  @Override
+  public void seek(long desired) throws IOException {
+    try {
+      super.seek(desired);
+    } catch (EOFException e) {
+      if (isGCSFileSystem) {

Review comment:
       I think it is really worth adding a comment here why we do such fallback 
seek, that it is only applicable for the last byte for GCS.

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java
##########
@@ -75,13 +76,13 @@
   public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema 
readerSchema, int bufferSize,
                              boolean readBlockLazily, boolean reverseReader) 
throws IOException {
     FSDataInputStream fsDataInputStream = fs.open(logFile.getPath(), 
bufferSize);
-    if (FSUtils.isGCSInputStream(fsDataInputStream)) {
-      this.inputStream = new TimedFSDataInputStream(logFile.getPath(), new 
FSDataInputStream(
-          new BufferedFSInputStream((FSInputStream) ((
-              (FSDataInputStream) 
fsDataInputStream.getWrappedStream()).getWrappedStream()), bufferSize)));
-    } else if (fsDataInputStream.getWrappedStream() instanceof FSInputStream) {
+    if (fsDataInputStream.getWrappedStream() instanceof FSInputStream) {
       this.inputStream = new TimedFSDataInputStream(logFile.getPath(), new 
FSDataInputStream(
           new BufferedFSInputStream((FSInputStream) 
fsDataInputStream.getWrappedStream(), bufferSize)));
+    } else if (FSUtils.isGCSFileSystem(fs)) {

Review comment:
       This piece of logic still slightly worries me a bit.
   
   I would also check fsDataInputStream.getWrappedStream().getWrappedStream() 
could be cast to FSInputStream and fallback to 'this.inputStream = 
fsDataInputStream' if it can't be. But I also would log a warning in this case 
as it would be unexpected situation that we may want to dig deeper in the 
future.
   
   thoughts?




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