-
Notifications
You must be signed in to change notification settings - Fork 1.5k
PARQUET-1822: Avoid requiring Hadoop installation for reading/writing #1111
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
Merged
Changes from all commits
Commits
Show all changes
19 commits
Select commit
Hold shift + click to select a range
6cafabf
PARQUET-1822: Add nio Path wrappers
amousavigourabi f41fc15
Add more documentation
amousavigourabi 3a81d6b
Add read/write tests with disk impl
amousavigourabi 0cc76f1
Add license headers
amousavigourabi 26ceb05
Fix reader test
amousavigourabi 710f0dd
Remove Javadoc
amousavigourabi a859954
Rename Disk -> Local
amousavigourabi 5a9a4de
Update Javadoc
amousavigourabi 0775c49
Update parquet-common/src/main/java/org/apache/parquet/io/LocalInputF…
amousavigourabi a200284
Update parquet-common/src/main/java/org/apache/parquet/io/LocalOutput…
amousavigourabi 7643868
Update tests disk -> local
amousavigourabi a5d5444
Resolves issues regarding create/overwrite
amousavigourabi e5ac8b6
Adds tests for create/overwrite
amousavigourabi b38c84a
Adds license header to test file
amousavigourabi ae69dd0
Merge branch 'apache:master' into avoid-hadoop-path
amousavigourabi 9f309a2
Use temp files for tests
amousavigourabi 223e7dd
Parameterizes tests
amousavigourabi 912547d
Update test
amousavigourabi 2ed43c6
Fix parameterization
amousavigourabi File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
102 changes: 102 additions & 0 deletions
102
parquet-common/src/main/java/org/apache/parquet/io/LocalInputFile.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,102 @@ | ||
| /* | ||
| * 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 LocalInputFile} 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; | ||
| private long length = -1; | ||
|
|
||
| public LocalInputFile(Path file) { | ||
| path = file; | ||
| } | ||
|
|
||
| @Override | ||
| public long getLength() throws IOException { | ||
| if (length == -1) { | ||
| try (RandomAccessFile file = new RandomAccessFile(path.toFile(), "r")) { | ||
| length = file.length(); | ||
| } | ||
| } | ||
| return length; | ||
| } | ||
|
|
||
| @Override | ||
| public SeekableInputStream newStream() throws IOException { | ||
|
|
||
| return new SeekableInputStream() { | ||
|
|
||
| private final RandomAccessFile randomAccessFile = new RandomAccessFile(path.toFile(), "r"); | ||
|
|
||
| @Override | ||
| public int read() throws IOException { | ||
| return randomAccessFile.read(); | ||
| } | ||
|
|
||
| @Override | ||
| public long getPos() throws IOException { | ||
| return randomAccessFile.getFilePointer(); | ||
| } | ||
|
|
||
| @Override | ||
| public void seek(long newPos) throws IOException { | ||
| randomAccessFile.seek(newPos); | ||
| } | ||
|
|
||
| @Override | ||
| public void readFully(byte[] bytes) throws IOException { | ||
| randomAccessFile.readFully(bytes); | ||
| } | ||
|
|
||
| @Override | ||
| public void readFully(byte[] bytes, int start, int len) throws IOException { | ||
| randomAccessFile.readFully(bytes, start, len); | ||
| } | ||
|
|
||
| @Override | ||
| public int read(ByteBuffer buf) throws IOException { | ||
| byte[] buffer = new byte[buf.remaining()]; | ||
| int code = read(buffer); | ||
| buf.put(buffer, buf.position() + buf.arrayOffset(), buf.remaining()); | ||
| return code; | ||
| } | ||
|
|
||
| @Override | ||
| public void readFully(ByteBuffer buf) throws IOException { | ||
| byte[] buffer = new byte[buf.remaining()]; | ||
| readFully(buffer); | ||
| buf.put(buffer, buf.position() + buf.arrayOffset(), buf.remaining()); | ||
| } | ||
|
|
||
| @Override | ||
| public void close() throws IOException { | ||
| randomAccessFile.close(); | ||
| } | ||
| }; | ||
| } | ||
| } |
107 changes: 107 additions & 0 deletions
107
parquet-common/src/main/java/org/apache/parquet/io/LocalOutputFile.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,107 @@ | ||
| /* | ||
| * 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; | ||
| import java.nio.file.StandardOpenOption; | ||
|
|
||
| /** | ||
| * {@code LocalOutputFile} is an implementation needed by Parquet to write | ||
| * to local data files using {@link PositionOutputStream} instances. | ||
| */ | ||
| public class LocalOutputFile implements OutputFile { | ||
|
|
||
| private class LocalPositionOutputStream extends PositionOutputStream { | ||
|
|
||
| private final BufferedOutputStream stream; | ||
| private long pos = 0; | ||
|
|
||
| public LocalPositionOutputStream(int buffer, StandardOpenOption... openOption) throws IOException { | ||
| stream = new BufferedOutputStream(Files.newOutputStream(path, openOption), buffer); | ||
| } | ||
|
|
||
| @Override | ||
| public long getPos() { | ||
| return pos; | ||
| } | ||
|
|
||
| @Override | ||
| public void write(int data) throws IOException { | ||
| pos++; | ||
wgtmac marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| stream.write(data); | ||
| } | ||
|
|
||
| @Override | ||
| public void write(byte[] data) throws IOException { | ||
| pos += data.length; | ||
| stream.write(data); | ||
| } | ||
|
|
||
| @Override | ||
| public void write(byte[] data, int off, int len) throws IOException { | ||
| pos += len; | ||
| stream.write(data, off, len); | ||
| } | ||
|
|
||
| @Override | ||
| public void flush() throws IOException { | ||
| stream.flush(); | ||
| } | ||
|
|
||
| @Override | ||
| public void close() throws IOException { | ||
| stream.close(); | ||
| } | ||
| } | ||
|
|
||
| private final Path path; | ||
|
|
||
| public LocalOutputFile(Path file) { | ||
| path = file; | ||
| } | ||
|
|
||
| @Override | ||
| public PositionOutputStream create(long buffer) throws IOException { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Could you please add a test case for |
||
| return new LocalPositionOutputStream((int) buffer, StandardOpenOption.CREATE_NEW); | ||
| } | ||
|
|
||
| @Override | ||
| public PositionOutputStream createOrOverwrite(long buffer) throws IOException { | ||
| return new LocalPositionOutputStream((int) buffer, StandardOpenOption.CREATE, | ||
| StandardOpenOption.TRUNCATE_EXISTING); | ||
| } | ||
|
|
||
| @Override | ||
| public boolean supportsBlockSize() { | ||
| return true; | ||
| } | ||
|
|
||
| @Override | ||
| public long defaultBlockSize() { | ||
| return 512; | ||
wgtmac marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| } | ||
|
|
||
| @Override | ||
| public String getPath() { | ||
| return path.toString(); | ||
| } | ||
| } | ||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.