You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@parquet.apache.org by "wgtmac (via GitHub)" <gi...@apache.org> on 2023/06/15 15:06:44 UTC

[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1111: PARQUET-1822: Avoid requiring Hadoop installation for reading/writing

wgtmac commented on code in PR #1111:
URL: https://github.com/apache/parquet-mr/pull/1111#discussion_r1231137142


##########
parquet-common/src/main/java/org/apache/parquet/io/LocalInputFile.java:
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.parquet.io;
+
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+
+/**
+ * {@code DiskInputFile} is an implementation needed by Parquet to read

Review Comment:
   ```suggestion
    * {@code LocalInputFile} is an implementation needed by Parquet to read
   ```



##########
parquet-common/src/main/java/org/apache/parquet/io/LocalOutputFile.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.parquet.io;
+
+import java.io.BufferedOutputStream;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+
+/**
+ * {@code DiskOutputFile} is an implementation needed by Parquet to write
+ * to local data files using {@link PositionOutputStream} instances.
+ */
+public class LocalOutputFile implements OutputFile {
+
+  private final Path path;
+
+  public LocalOutputFile(Path file) {
+    path = file;
+  }
+
+  @Override
+  public PositionOutputStream create(long buffer) throws IOException {
+    return new PositionOutputStream() {
+
+      private final BufferedOutputStream stream =
+        new BufferedOutputStream(Files.newOutputStream(path), (int) buffer);

Review Comment:
   Does this support overwrite?



##########
parquet-common/src/main/java/org/apache/parquet/io/LocalOutputFile.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.parquet.io;
+
+import java.io.BufferedOutputStream;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+
+/**
+ * {@code DiskOutputFile} is an implementation needed by Parquet to write

Review Comment:
   ```suggestion
    * {@code LocalOutputFile} is an implementation needed by Parquet to write
   ```



##########
parquet-common/src/main/java/org/apache/parquet/io/LocalOutputFile.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.parquet.io;
+
+import java.io.BufferedOutputStream;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+
+/**
+ * {@code DiskOutputFile} is an implementation needed by Parquet to write
+ * to local data files using {@link PositionOutputStream} instances.
+ */
+public class LocalOutputFile implements OutputFile {
+
+  private final Path path;
+
+  public LocalOutputFile(Path file) {
+    path = file;
+  }
+
+  @Override
+  public PositionOutputStream create(long buffer) throws IOException {
+    return new PositionOutputStream() {
+
+      private final BufferedOutputStream stream =
+        new BufferedOutputStream(Files.newOutputStream(path), (int) buffer);

Review Comment:
   I would expect `create` fails if file with same name exists.



##########
parquet-common/src/main/java/org/apache/parquet/io/LocalInputFile.java:
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.parquet.io;
+
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+
+/**
+ * {@code DiskInputFile} is an implementation needed by Parquet to read
+ * from local data files using {@link SeekableInputStream} instances.
+ */
+public class LocalInputFile implements InputFile {
+
+  private final Path path;
+
+  public LocalInputFile(Path file) {
+    path = file;
+  }
+
+  @Override
+  public long getLength() throws IOException {
+    RandomAccessFile file = new RandomAccessFile(path.toFile(), "r");
+    long length = file.length();

Review Comment:
   Should it be cached in case of repeated read?
   
   Or would `path.toFile().length()` do the same thing?



-- 
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: dev-unsubscribe@parquet.apache.org

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