You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/05/31 13:06:04 UTC

[GitHub] [iceberg] nastra opened a new pull request, #4918: Core: Add InMemoryFileIO implementation

nastra opened a new pull request, #4918:
URL: https://github.com/apache/iceberg/pull/4918

   This is a revival of https://github.com/apache/iceberg/pull/4054 to get this over the line. I also addressed review feedback and improved/changed a few other things.


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] snazy commented on a diff in pull request #4918: Core: Add InMemoryFileIO implementation

Posted by GitBox <gi...@apache.org>.
snazy commented on code in PR #4918:
URL: https://github.com/apache/iceberg/pull/4918#discussion_r885645044


##########
core/src/main/java/org/apache/iceberg/io/inmemory/InMemoryFileStore.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.iceberg.io.inmemory;
+
+import java.nio.ByteBuffer;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.ByteBuffers;
+
+/** An in-memory collection based storage for file contents keyed by a string location. */
+class InMemoryFileStore {
+
+  private final ConcurrentMap<String, byte[]> store;
+
+  InMemoryFileStore() {
+    this.store = Maps.newConcurrentMap();
+  }
+
+  /** Put the file contents at the given location, overwrite if it already exists. */
+  public void put(String location, ByteBuffer data) {
+    // Copy the contents and store it.
+    store.put(location, ByteBuffers.toByteArray(data));
+  }
+
+  /** Get the file contents for the given location. */
+  public Optional<ByteBuffer> get(String location) {
+    return Optional.ofNullable(store.get(location))
+        .map(
+            bytes -> {

Review Comment:
   You can store the `ByteBuffer` in `put()` using `ByteBuffer.asReadOnlyBuffer()` and then return just return `ByteBuffer.duplicate()` here instead of the byte[] copy + wrapping.



##########
core/src/main/java/org/apache/iceberg/io/inmemory/InMemoryFileStore.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.iceberg.io.inmemory;
+
+import java.nio.ByteBuffer;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.ByteBuffers;
+
+/** An in-memory collection based storage for file contents keyed by a string location. */
+class InMemoryFileStore {

Review Comment:
   Maybe move the code from this class into `InMemoryFileIO` and make `InMemoryFileIO`'s inner classes top-level classes?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra closed pull request #4918: Core: Add InMemoryFileIO implementation

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra closed pull request #4918: Core: Add InMemoryFileIO implementation
URL: https://github.com/apache/iceberg/pull/4918


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #4918: Core: Add InMemoryFileIO implementation

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4918:
URL: https://github.com/apache/iceberg/pull/4918#discussion_r912159458


##########
core/src/test/java/org/apache/iceberg/avro/TestAvroDeleteWriters.java:
##########
@@ -74,7 +75,7 @@ public void createDeleteRecords() {
 
   @Test
   public void testEqualityDeleteWriter() throws IOException {
-    OutputFile out = new InMemoryOutputFile();
+    OutputFile out = new InMemoryFileIO().newOutputFile(UUID.randomUUID().toString());

Review Comment:
   Same comment here about saving and reusing the `InMemoryFileIO`, especially as this class uses it multiple times.



##########
core/src/main/java/org/apache/iceberg/io/inmemory/InMemoryFileIO.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.iceberg.io.inmemory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class InMemoryFileIO implements FileIO {
+  private final InMemoryFileStore store;
+
+  public InMemoryFileIO() {
+    store = new InMemoryFileStore();
+  }
+
+  InMemoryFileStore getStore() {

Review Comment:
   Nit: We tend not to use `get` in Iceberg. The logic behind that being that there’s either a more specific verb available (like `fetch` etc) or that the `get` is simply unneeded.
   
   In this case for an accessor I think it’s simply not needed.



##########
core/src/test/java/org/apache/iceberg/TestScansAndSchemaEvolution.java:
##########
@@ -70,7 +70,8 @@ public TestScansAndSchemaEvolution(int formatVersion) {
   private DataFile createDataFile(String partValue) throws IOException {
     List<GenericData.Record> expected = RandomAvroData.generate(SCHEMA, 100, 0L);
 
-    OutputFile dataFile = new InMemoryOutputFile(FileFormat.AVRO.addExtension(UUID.randomUUID().toString()));
+    OutputFile dataFile =
+        new InMemoryFileIO().newOutputFile(FileFormat.AVRO.addExtension(UUID.randomUUID().toString()));

Review Comment:
   Should we store the `InMemoryFileIO` in its own variable so that the underlying `store` is only instantiated once?



##########
core/src/test/java/org/apache/iceberg/io/inmemory/InMemoryFileIOTest.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.iceberg.io.inmemory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.stream.IntStream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.assertj.core.api.Assertions;
+import org.assertj.core.data.Offset;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+public class InMemoryFileIOTest {
+
+  private InMemoryFileIO fileIO;
+  private InMemoryFileStore store;
+
+  @BeforeEach
+  public void before() {
+    fileIO = new InMemoryFileIO();
+    store = fileIO.getStore();
+  }
+
+  @Test
+  public void testGetStore() {
+    Assertions.assertThat(store).isNotNull();
+  }
+
+  @Test
+  public void testDeleteFile() {
+    Assertions.assertThat(store.exists("file1")).isFalse();
+
+    // Create the file
+    store.put("file1", ByteBuffer.wrap(new byte[0]));
+    Assertions.assertThat(store.exists("file1")).isTrue();
+
+    // Delete the file
+    fileIO.deleteFile("file1");
+    // Verify that the file has been deleted
+    Assertions.assertThat(store.exists("file1")).isFalse();
+
+    Assertions.assertThatThrownBy(() -> fileIO.deleteFile("nonExisting"))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("File at location 'nonExisting' does not exist");
+  }
+
+  @Test
+  public void testNewInputFile() throws IOException {
+    String fileName = "file1";
+
+    store.put(fileName, ByteBuffer.wrap("data1".getBytes(UTF_8)));
+
+    InputFile inputFile = fileIO.newInputFile(fileName);
+    Assertions.assertThat(inputFile).isNotNull();
+
+    Assertions.assertThat(inputFile.exists()).isTrue();
+    Assertions.assertThat(inputFile.location()).isEqualTo(fileName);
+    Assertions.assertThat(inputFile.getLength()).isEqualTo(5);
+
+    try (SeekableInputStream inputStream = inputFile.newStream()) {
+      Assertions.assertThat(inputStream).isNotNull();
+
+      // Test read()
+      Assertions.assertThat(inputStream.getPos()).isEqualTo(0);
+      Assertions.assertThat(inputStream.read()).isEqualTo('d');
+
+      // Test read(byte[], index, len)
+      byte[] dataRead = new byte[3];
+      Assertions.assertThat(inputStream.read(dataRead, 0, 2)).isEqualTo(2);
+      Assertions.assertThat(new String(dataRead, 0, 2, UTF_8)).isEqualTo("at");
+    }
+  }
+
+  @Test
+  public void testSeek() throws IOException {
+    String fileName = "file1";
+
+    // Number of rows
+    int numRowsInChunk = 10;
+
+    // Emulate parquet file structure
+    ByteBuffer buffer = ByteBuffer.allocate(1024);
+    // Add magic number
+    buffer.put("par1".getBytes(UTF_8));
+    // Add column chunk 1 for int32
+    IntStream.range(0, numRowsInChunk).forEach(buffer::putInt);
+    // Add column chunk 2 for float64
+    IntStream.range(0, numRowsInChunk).forEach(buffer::putDouble);
+    // Add metadata footer (index to the start of chunks)
+    buffer.putInt(4);
+    buffer.putInt(4 + numRowsInChunk * 4);
+    // Add footer size
+    buffer.putInt(8);
+    // Add magic number
+    buffer.put("par1".getBytes(UTF_8));
+
+    // Put the data in the store
+    buffer.flip();
+    store.put(fileName, buffer);
+
+    // magic number, chunk1, chunk2, footer, footer size, magic number
+    int expectedFileSize = 4 + numRowsInChunk * 4 + numRowsInChunk * 8 + 8 + 4 + 4;

Review Comment:
   Nit: Some of these might be better off as constants. But since it's all in the same method and we'll defined above I'm ok either way.



##########
core/src/main/java/org/apache/iceberg/io/inmemory/InMemoryFileIO.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.iceberg.io.inmemory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.ByteBuffers;
+
+public class InMemoryFileIO implements FileIO {
+  private final InMemoryFileStore store;
+
+  public InMemoryFileIO() {
+    store = new InMemoryFileStore();
+  }
+
+  InMemoryFileStore getStore() {
+    return store;
+  }
+
+  @Override
+  public InputFile newInputFile(String path) {
+    return new InMemoryInputFile(path);
+  }
+
+  @Override
+  public OutputFile newOutputFile(String path) {
+    return new InMemoryOutputFile(path);
+  }
+
+  @Override
+  public void deleteFile(String path) {
+    Preconditions.checkArgument(store.remove(path), "File at location '%s' does not exist", path);
+  }
+
+  private class InMemoryInputFile implements InputFile {
+    private final String location;
+
+    InMemoryInputFile(String location) {
+      this.location = location;
+    }
+
+    @Override
+    public long getLength() {
+      return getOrThrow().remaining();
+    }
+
+    @Override
+    public SeekableInputStream newStream() {
+      return new InMemorySeekableInputStream(getOrThrow().duplicate());
+    }
+
+    @Override
+    public String location() {
+      return location;
+    }
+
+    @Override
+    public boolean exists() {
+      return store.exists(location);
+    }
+
+    private ByteBuffer getOrThrow() {

Review Comment:
   In this case, given that the `get` clearly lines up with the existing language and its private I feel it’s fine. 👍



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #4918: Core: Add InMemoryFileIO implementation

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #4918:
URL: https://github.com/apache/iceberg/pull/4918#discussion_r885699460


##########
core/src/main/java/org/apache/iceberg/io/inmemory/InMemoryFileStore.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.iceberg.io.inmemory;
+
+import java.nio.ByteBuffer;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.ByteBuffers;
+
+/** An in-memory collection based storage for file contents keyed by a string location. */
+class InMemoryFileStore {
+
+  private final ConcurrentMap<String, byte[]> store;
+
+  InMemoryFileStore() {
+    this.store = Maps.newConcurrentMap();
+  }
+
+  /** Put the file contents at the given location, overwrite if it already exists. */
+  public void put(String location, ByteBuffer data) {
+    // Copy the contents and store it.
+    store.put(location, ByteBuffers.toByteArray(data));
+  }
+
+  /** Get the file contents for the given location. */
+  public Optional<ByteBuffer> get(String location) {
+    return Optional.ofNullable(store.get(location))
+        .map(
+            bytes -> {

Review Comment:
   good point, done



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on pull request #4918: Core: Add InMemoryFileIO implementation

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on PR #4918:
URL: https://github.com/apache/iceberg/pull/4918#issuecomment-1443106547

   Closing this as we already have InMemoryFileIO in the codebase


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #4918: Core: Add InMemoryFileIO implementation

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4918:
URL: https://github.com/apache/iceberg/pull/4918#discussion_r912162163


##########
core/src/test/java/org/apache/iceberg/io/inmemory/InMemoryFileIOTest.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.iceberg.io.inmemory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.stream.IntStream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.assertj.core.api.Assertions;
+import org.assertj.core.data.Offset;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+public class InMemoryFileIOTest {
+
+  private InMemoryFileIO fileIO;
+  private InMemoryFileStore store;
+
+  @BeforeEach
+  public void before() {
+    fileIO = new InMemoryFileIO();
+    store = fileIO.getStore();
+  }
+
+  @Test
+  public void testGetStore() {
+    Assertions.assertThat(store).isNotNull();
+  }
+
+  @Test
+  public void testDeleteFile() {
+    Assertions.assertThat(store.exists("file1")).isFalse();
+
+    // Create the file
+    store.put("file1", ByteBuffer.wrap(new byte[0]));
+    Assertions.assertThat(store.exists("file1")).isTrue();
+
+    // Delete the file
+    fileIO.deleteFile("file1");
+    // Verify that the file has been deleted
+    Assertions.assertThat(store.exists("file1")).isFalse();
+
+    Assertions.assertThatThrownBy(() -> fileIO.deleteFile("nonExisting"))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("File at location 'nonExisting' does not exist");
+  }
+
+  @Test
+  public void testNewInputFile() throws IOException {
+    String fileName = "file1";
+
+    store.put(fileName, ByteBuffer.wrap("data1".getBytes(UTF_8)));
+
+    InputFile inputFile = fileIO.newInputFile(fileName);
+    Assertions.assertThat(inputFile).isNotNull();
+
+    Assertions.assertThat(inputFile.exists()).isTrue();
+    Assertions.assertThat(inputFile.location()).isEqualTo(fileName);
+    Assertions.assertThat(inputFile.getLength()).isEqualTo(5);
+
+    try (SeekableInputStream inputStream = inputFile.newStream()) {
+      Assertions.assertThat(inputStream).isNotNull();
+
+      // Test read()
+      Assertions.assertThat(inputStream.getPos()).isEqualTo(0);
+      Assertions.assertThat(inputStream.read()).isEqualTo('d');
+
+      // Test read(byte[], index, len)
+      byte[] dataRead = new byte[3];
+      Assertions.assertThat(inputStream.read(dataRead, 0, 2)).isEqualTo(2);
+      Assertions.assertThat(new String(dataRead, 0, 2, UTF_8)).isEqualTo("at");
+    }
+  }
+
+  @Test
+  public void testSeek() throws IOException {
+    String fileName = "file1";
+
+    // Number of rows
+    int numRowsInChunk = 10;
+
+    // Emulate parquet file structure
+    ByteBuffer buffer = ByteBuffer.allocate(1024);
+    // Add magic number
+    buffer.put("par1".getBytes(UTF_8));
+    // Add column chunk 1 for int32
+    IntStream.range(0, numRowsInChunk).forEach(buffer::putInt);
+    // Add column chunk 2 for float64
+    IntStream.range(0, numRowsInChunk).forEach(buffer::putDouble);
+    // Add metadata footer (index to the start of chunks)
+    buffer.putInt(4);
+    buffer.putInt(4 + numRowsInChunk * 4);
+    // Add footer size
+    buffer.putInt(8);
+    // Add magic number
+    buffer.put("par1".getBytes(UTF_8));
+
+    // Put the data in the store
+    buffer.flip();
+    store.put(fileName, buffer);
+
+    // magic number, chunk1, chunk2, footer, footer size, magic number
+    int expectedFileSize = 4 + numRowsInChunk * 4 + numRowsInChunk * 8 + 8 + 4 + 4;

Review Comment:
   Nit: Some of these might be better off as constants. But since it's all in the same method and well defined above I'm ok either way.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org