You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by xu...@apache.org on 2022/10/17 04:44:56 UTC

[hudi] branch master updated: [HUDI-5003] Fix the type of InLineFileSystem`startOffset to long (#6916)

This is an automated email from the ASF dual-hosted git repository.

xushiyan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hudi.git


The following commit(s) were added to refs/heads/master by this push:
     new 76f3c6af03 [HUDI-5003] Fix the type of InLineFileSystem`startOffset to long (#6916)
76f3c6af03 is described below

commit 76f3c6af03f637007896b80c31874ac2f462025e
Author: wulei <wu...@bytedance.com>
AuthorDate: Mon Oct 17 12:44:50 2022 +0800

    [HUDI-5003] Fix the type of InLineFileSystem`startOffset to long (#6916)
---
 .../hudi/common/fs/inline/InLineFSUtils.java       |  8 +--
 .../common/fs/inline/InLineFsDataInputStream.java  |  6 +--
 .../hudi/common/fs/inline/InLineFSUtilsTest.java   | 59 ++++++++++++++++++++++
 3 files changed, 66 insertions(+), 7 deletions(-)

diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/inline/InLineFSUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/inline/InLineFSUtils.java
index 080f228f16..6031f29d90 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/fs/inline/InLineFSUtils.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/inline/InLineFSUtils.java
@@ -95,11 +95,11 @@ public class InLineFSUtils {
    * input: "inlinefs://file1/s3a/?start_offset=20&length=40".
    * output: 20
    */
-  public static int startOffset(Path inlineFSPath) {
+  public static long startOffset(Path inlineFSPath) {
     assertInlineFSPath(inlineFSPath);
 
     String[] slices = inlineFSPath.toString().split("[?&=]");
-    return Integer.parseInt(slices[slices.length - 3]);
+    return Long.parseLong(slices[slices.length - 3]);
   }
 
   /**
@@ -108,11 +108,11 @@ public class InLineFSUtils {
    * input: "inlinefs:/file1/s3a/?start_offset=20&length=40".
    * output: 40
    */
-  public static int length(Path inlinePath) {
+  public static long length(Path inlinePath) {
     assertInlineFSPath(inlinePath);
 
     String[] slices = inlinePath.toString().split("[?&=]");
-    return Integer.parseInt(slices[slices.length - 1]);
+    return Long.parseLong(slices[slices.length - 1]);
   }
 
   private static void assertInlineFSPath(Path inlinePath) {
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/inline/InLineFsDataInputStream.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/inline/InLineFsDataInputStream.java
index 4e8701244c..fbd067c6c1 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/fs/inline/InLineFsDataInputStream.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/inline/InLineFsDataInputStream.java
@@ -33,11 +33,11 @@ import java.util.EnumSet;
  */
 public class InLineFsDataInputStream extends FSDataInputStream {
 
-  private final int startOffset;
+  private final long startOffset;
   private final FSDataInputStream outerStream;
-  private final int length;
+  private final long length;
 
-  public InLineFsDataInputStream(int startOffset, FSDataInputStream outerStream, int length) throws IOException {
+  public InLineFsDataInputStream(long startOffset, FSDataInputStream outerStream, long length) throws IOException {
     super(outerStream.getWrappedStream());
     this.startOffset = startOffset;
     this.outerStream = outerStream;
diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/InLineFSUtilsTest.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/InLineFSUtilsTest.java
new file mode 100644
index 0000000000..7d704c9112
--- /dev/null
+++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/InLineFSUtilsTest.java
@@ -0,0 +1,59 @@
+/*
+ * 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.inline;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.testutils.FileSystemTestUtils;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/**
+ * Tests {@link InLineFileSystem}.
+ */
+public class InLineFSUtilsTest {
+
+  private static Stream<Arguments> configParams() {
+    Long[] data = new Long[] {
+        0L,
+        1000L,
+        (long) Integer.MAX_VALUE + 1,
+        Long.MAX_VALUE
+    };
+    return Stream.of(data).map(Arguments::of);
+  }
+
+  @ParameterizedTest
+  @MethodSource("configParams")
+  void startOffset(long startOffset) {
+    Path inlinePath =  FileSystemTestUtils.getPhantomFile(FileSystemTestUtils.getRandomOuterFSPath(), startOffset, 0L);
+    assertEquals(startOffset, InLineFSUtils.startOffset(inlinePath));
+  }
+
+  @ParameterizedTest
+  @MethodSource("configParams")
+  void length(long inlineLength) {
+    Path inlinePath =  FileSystemTestUtils.getPhantomFile(FileSystemTestUtils.getRandomOuterFSPath(), 0L, inlineLength);
+    assertEquals(inlineLength, InLineFSUtils.length(inlinePath));
+  }
+}