kbendick commented on code in PR #4534:
URL: https://github.com/apache/iceberg/pull/4534#discussion_r850610082


##########
core/src/test/java/org/apache/iceberg/io/InMemoryInputFile.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.Objects;
+import java.util.UUID;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class InMemoryInputFile implements InputFile {
+
+  private final String location;
+  private final byte[] contents;
+
+  public InMemoryInputFile(byte[] contents) {
+    this("memory:" + UUID.randomUUID(), contents);
+  }
+
+  public InMemoryInputFile(String location, byte[] contents) {
+    this.location = Objects.requireNonNull(location, "location is null");
+    this.contents = Objects.requireNonNull(contents, "contents is 
null").clone();
+  }
+
+  @Override
+  public long getLength() {
+    return contents.length;
+  }
+
+  @Override
+  public SeekableInputStream newStream() {
+    return new InMemorySeekableInputStream(contents);
+  }
+
+  @Override
+  public String location() {
+    return location;
+  }
+
+  @Override
+  public boolean exists() {
+    return true;
+  }
+
+  private static class InMemorySeekableInputStream extends SeekableInputStream 
{
+
+    private final int length;
+    private final ByteArrayInputStream delegate;
+
+    InMemorySeekableInputStream(byte[] contents) {
+      this.length = contents.length;
+      this.delegate = new ByteArrayInputStream(contents);
+    }
+
+    @Override
+    public long getPos() throws IOException {
+      return length - delegate.available();
+    }
+
+    @Override
+    public void seek(long newPos) throws IOException {
+      delegate.reset();
+      Preconditions.checkState(delegate.skip(newPos) == newPos,
+          "Invalid position %s within stream of length %s", newPos, length);
+    }
+
+    @Override
+    public int read() {
+      return delegate.read();
+    }
+
+    @Override
+    public int read(byte[] b) throws IOException {
+      return delegate.read(b);
+    }
+
+    @Override
+    public int read(byte[] b, int off, int len) {
+      return delegate.read(b, off, len);
+    }
+
+    @Override
+    public long skip(long n) {
+      return delegate.skip(n);
+    }
+
+    @Override
+    public int available() {
+      return delegate.available();
+    }
+
+    @Override
+    public boolean markSupported() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void mark(int readAheadLimit) {
+      // We use mark to implement seek

Review Comment:
   Nit: I think this comment is out of date? It doesn't seem that we use `mark` 
for anything.



##########
core/src/test/java/org/apache/iceberg/io/InMemoryOutputFile.java:
##########
@@ -0,0 +1,108 @@
+/*
+ * 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;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.Objects;
+import java.util.UUID;
+
+public class InMemoryOutputFile implements OutputFile {
+
+  private final String location;
+
+  private boolean exists;
+  private ByteArrayOutputStream contents;
+
+  public InMemoryOutputFile() {
+    this("memory:" + UUID.randomUUID());
+  }
+
+  public InMemoryOutputFile(String location) {
+    this.location = Objects.requireNonNull(location, "location is null");
+  }
+
+  @Override
+  public PositionOutputStream create() {
+    if (exists) {
+      throw new RuntimeException("Already exists");
+    }
+    return createOrOverwrite();
+  }
+
+  @Override
+  public PositionOutputStream createOrOverwrite() {
+    exists = true;
+    contents = new ByteArrayOutputStream();
+    return new InMemoryPositionOutputStream(contents);
+  }
+
+  @Override
+  public String location() {
+    return location;
+  }
+
+  @Override
+  public InputFile toInputFile() {
+    return new InMemoryInputFile(location(), getContents());
+  }
+
+  public byte[] getContents() {
+    return contents.toByteArray();
+  }

Review Comment:
   Nit: Usually we don't use `get` in front of accessors. Either there's a more 
descriptive verb for the situation (e.g. something like `find`) or we just put 
the name directly.
   
   So in this case, we'd use `contents()` for the accessor.



##########
core/src/test/java/org/apache/iceberg/io/InMemoryInputFile.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.Objects;
+import java.util.UUID;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class InMemoryInputFile implements InputFile {
+
+  private final String location;
+  private final byte[] contents;
+
+  public InMemoryInputFile(byte[] contents) {
+    this("memory:" + UUID.randomUUID(), contents);
+  }
+
+  public InMemoryInputFile(String location, byte[] contents) {
+    this.location = Objects.requireNonNull(location, "location is null");
+    this.contents = Objects.requireNonNull(contents, "contents is 
null").clone();

Review Comment:
   Usually we use `Preconditions.checkNotNull`, but since this code is in test 
the style rules are somewhat more lax.



##########
core/src/test/java/org/apache/iceberg/io/InMemoryInputFile.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.Objects;
+import java.util.UUID;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class InMemoryInputFile implements InputFile {
+
+  private final String location;
+  private final byte[] contents;
+
+  public InMemoryInputFile(byte[] contents) {
+    this("memory:" + UUID.randomUUID(), contents);
+  }
+
+  public InMemoryInputFile(String location, byte[] contents) {
+    this.location = Objects.requireNonNull(location, "location is null");
+    this.contents = Objects.requireNonNull(contents, "contents is 
null").clone();
+  }
+
+  @Override
+  public long getLength() {
+    return contents.length;
+  }
+
+  @Override
+  public SeekableInputStream newStream() {
+    return new InMemorySeekableInputStream(contents);
+  }
+
+  @Override
+  public String location() {
+    return location;
+  }
+
+  @Override
+  public boolean exists() {
+    return true;
+  }
+
+  private static class InMemorySeekableInputStream extends SeekableInputStream 
{
+
+    private final int length;
+    private final ByteArrayInputStream delegate;
+
+    InMemorySeekableInputStream(byte[] contents) {
+      this.length = contents.length;
+      this.delegate = new ByteArrayInputStream(contents);
+    }
+
+    @Override
+    public long getPos() throws IOException {
+      return length - delegate.available();
+    }
+
+    @Override
+    public void seek(long newPos) throws IOException {
+      delegate.reset();
+      Preconditions.checkState(delegate.skip(newPos) == newPos,
+          "Invalid position %s within stream of length %s", newPos, length);
+    }
+
+    @Override
+    public int read() {
+      return delegate.read();
+    }
+
+    @Override
+    public int read(byte[] b) throws IOException {
+      return delegate.read(b);
+    }
+
+    @Override
+    public int read(byte[] b, int off, int len) {
+      return delegate.read(b, off, len);
+    }
+
+    @Override
+    public long skip(long n) {
+      return delegate.skip(n);
+    }
+
+    @Override
+    public int available() {
+      return delegate.available();
+    }
+
+    @Override
+    public boolean markSupported() {
+      throw new UnsupportedOperationException();
+    }

Review Comment:
   Nit: Is there a benefit to throwing `UnsupportedOperationException` or 
should this return `false`?



##########
core/src/test/java/org/apache/iceberg/io/InMemoryOutputFile.java:
##########
@@ -0,0 +1,108 @@
+/*
+ * 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;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.Objects;
+import java.util.UUID;
+
+public class InMemoryOutputFile implements OutputFile {
+
+  private final String location;

Review Comment:
   The current `LocalOutputFile` returns the `location` in its `toString` 
method.
   
   I believe this is to match the behavior of `file.toString()`, which might be 
relied on elsewhere.
   
   Should we override it here as well so the two are more consistent?
   
   
https://github.com/apache/iceberg/blob/e45c19e1d02a130d33ac886f676dfb141708090d/api/src/main/java/org/apache/iceberg/Files.java#L94-L97



##########
core/src/test/java/org/apache/iceberg/io/InMemoryOutputFile.java:
##########
@@ -0,0 +1,108 @@
+/*
+ * 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;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.Objects;
+import java.util.UUID;
+
+public class InMemoryOutputFile implements OutputFile {
+
+  private final String location;
+
+  private boolean exists;
+  private ByteArrayOutputStream contents;
+
+  public InMemoryOutputFile() {
+    this("memory:" + UUID.randomUUID());
+  }
+
+  public InMemoryOutputFile(String location) {
+    this.location = Objects.requireNonNull(location, "location is null");
+  }
+
+  @Override
+  public PositionOutputStream create() {
+    if (exists) {
+      throw new RuntimeException("Already exists");

Review Comment:
   Can we throw `AlreadyExistsException` to match the current 
`LocalOutputFile#create()` method?
   
   
https://github.com/apache/iceberg/blob/e45c19e1d02a130d33ac886f676dfb141708090d/api/src/main/java/org/apache/iceberg/Files.java#L57-L59



##########
core/src/test/java/org/apache/iceberg/io/InMemoryInputFile.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.Objects;
+import java.util.UUID;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class InMemoryInputFile implements InputFile {
+
+  private final String location;
+  private final byte[] contents;
+
+  public InMemoryInputFile(byte[] contents) {
+    this("memory:" + UUID.randomUUID(), contents);
+  }
+
+  public InMemoryInputFile(String location, byte[] contents) {
+    this.location = Objects.requireNonNull(location, "location is null");
+    this.contents = Objects.requireNonNull(contents, "contents is 
null").clone();
+  }
+
+  @Override
+  public long getLength() {
+    return contents.length;
+  }
+
+  @Override
+  public SeekableInputStream newStream() {
+    return new InMemorySeekableInputStream(contents);
+  }
+
+  @Override
+  public String location() {
+    return location;
+  }
+
+  @Override
+  public boolean exists() {
+    return true;
+  }

Review Comment:
   Question: Should we be tracking `exists` (e.g. just in the constructor)? 
`LocalInputFile` does technically track `exists` in that the underlying file 
will return `false` if that file object is removed.
   
   The current usage of `InMemoryInputFIle` don't seem to need that, but I'm 
not sure about remaining usages of `LocalInputFile`. It might not be necessary.



##########
core/src/test/java/org/apache/iceberg/io/InMemoryInputFile.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.Objects;
+import java.util.UUID;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class InMemoryInputFile implements InputFile {
+
+  private final String location;
+  private final byte[] contents;
+
+  public InMemoryInputFile(byte[] contents) {
+    this("memory:" + UUID.randomUUID(), contents);
+  }
+
+  public InMemoryInputFile(String location, byte[] contents) {
+    this.location = Objects.requireNonNull(location, "location is null");
+    this.contents = Objects.requireNonNull(contents, "contents is 
null").clone();
+  }
+
+  @Override
+  public long getLength() {
+    return contents.length;
+  }
+
+  @Override
+  public SeekableInputStream newStream() {
+    return new InMemorySeekableInputStream(contents);
+  }
+
+  @Override
+  public String location() {
+    return location;
+  }
+
+  @Override
+  public boolean exists() {
+    return true;
+  }
+
+  private static class InMemorySeekableInputStream extends SeekableInputStream 
{
+
+    private final int length;
+    private final ByteArrayInputStream delegate;
+
+    InMemorySeekableInputStream(byte[] contents) {
+      this.length = contents.length;
+      this.delegate = new ByteArrayInputStream(contents);
+    }
+
+    @Override
+    public long getPos() throws IOException {
+      return length - delegate.available();
+    }
+
+    @Override
+    public void seek(long newPos) throws IOException {
+      delegate.reset();
+      Preconditions.checkState(delegate.skip(newPos) == newPos,
+          "Invalid position %s within stream of length %s", newPos, length);
+    }

Review Comment:
   Nit: We should consider adding an explicit check that the new position is 
non-negative.
   
   The `checkState` call seems like it will cover that as `skip` shouldn't 
return a negative number, but it might be better for tests to add an explicit 
check. WDYT?



##########
core/src/test/java/org/apache/iceberg/io/InMemoryOutputFile.java:
##########
@@ -0,0 +1,108 @@
+/*
+ * 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;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.Objects;
+import java.util.UUID;
+
+public class InMemoryOutputFile implements OutputFile {
+
+  private final String location;
+
+  private boolean exists;

Review Comment:
   Do we explicitly set `exists` to `false` anywhere or are we relying on the 
default value being false?
   
   I'd prefer we explicitly set the value false somewhere for readability 
purposes.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to