Skip to content
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

PARQUET-1822: Avoid requiring Hadoop installation for reading/writing #1111

Merged
merged 19 commits into from Jul 4, 2023

Conversation

amousavigourabi
Copy link
Contributor

Make sure you have checked all steps below.

Jira

Tests

  • My PR adds the following unit tests OR does not need testing for this extremely good reason:

Commits

  • My commits all reference Jira issues in their subject lines. In addition, my commits follow the guidelines from "How to write a good git commit message":
    1. Subject is separated from body by a blank line
    2. Subject is limited to 50 characters (not including Jira issue reference)
    3. Subject does not end with a period
    4. Subject uses the imperative mood ("add", not "adding")
    5. Body wraps at 72 characters
    6. Body explains "what" and "why", not "how"

Documentation

  • In case of new functionality, my PR adds documentation that describes how to use it.
    • All the public functions and the classes in the PR contain Javadoc that explain what it does

Add disk InputFile and OutputFile implementations
Add some Javadoc to OutputFile
@amousavigourabi amousavigourabi marked this pull request as ready for review June 12, 2023 20:46
@amousavigourabi amousavigourabi force-pushed the avoid-hadoop-path branch 5 times, most recently from 4fa92ba to 680b8d9 Compare June 13, 2023 06:57
@wernerdaehn
Copy link

I don't want to sound too greedy, but the next level of this feature would be if the classes in question have no imports of Hadoop in them.
Something like: Parquet(with Hadoop) -> Parquet(with java.nio.File)
And the lower level classes are a jar of their own.

Just dreaming...

@amousavigourabi
Copy link
Contributor Author

I don't want to sound too greedy, but the next level of this feature would be if the classes in question have no imports of Hadoop in them. Something like: Parquet(with Hadoop) -> Parquet(with java.nio.File) And the lower level classes are a jar of their own.

Just dreaming...

One day... The next step is to get rid of the tight coupling to the other Hadoop classes (mainly Configuration) as that shouldn't break anything, which would at least allow users to drop hadoop-client-runtime. But first, this PR is to allow users to avoid the bigger Hadoop issues more easily.

@wernerdaehn
Copy link

And I greatly appreciate your work!

@amousavigourabi amousavigourabi force-pushed the avoid-hadoop-path branch 2 times, most recently from cafa9d1 to 99c57ea Compare June 13, 2023 08:16
@wgtmac
Copy link
Member

wgtmac commented Jun 15, 2023

@gszadovszky @shangxinli @Fokko Do you have time to take a look? This has been discussed in the mailing list: https://lists.apache.org/thread/d33757j99xqn63hrfz415sq60v3x9hmy

Copy link
Contributor

@gszadovszky gszadovszky left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks a lot for working on this @amousavigourabi!

About the method comments. I would only add a method comment that overrides another if there are changes in the behavior comparing to the definition of the super one.

See comments made by @gszadovszky under apache#1111
Copy link
Contributor

@gszadovszky gszadovszky left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry @amousavigourabi, the comments related to the naming "disk" intended to be posted with my previous review. Somehow these comments got lost from it.

See comments made by @gszadovszky under apache#1111
@gszadovszky
Copy link
Contributor

@amousavigourabi, please, also update the class comments of LocalInputFile and LocalOutputFile accordingly.

See comments made by @gszadovszky under apache#1111
@Override
public long getLength() throws IOException {
RandomAccessFile file = new RandomAccessFile(path.toFile(), "r");
long length = file.length();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should it be cached in case of repeated read?

Or would path.toFile().length() do the same thing?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is stored in a long after the first read now.

return new PositionOutputStream() {

private final BufferedOutputStream stream =
new BufferedOutputStream(Files.newOutputStream(path), (int) buffer);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does this support overwrite?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would expect create fails if file with same name exists.

Copy link
Member

@wgtmac wgtmac left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @amousavigourabi! Comments are mostly for testing.

@@ -117,19 +120,52 @@ public void testEmptyArray() throws Exception {
}
}

@Test
public void testEmptyArrayLocal() throws Exception {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This duplicates testEmptyArray(). Can we parameterize them to share common code?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good idea, done

@@ -145,6 +181,39 @@ public void testEmptyMap() throws Exception {
}
}

@Test
public void testEmptyMapLocal() throws Exception {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ditto


public class TestLocalInputOutput {

Path pathFileExists = Paths.get("src/test/resources/disk_output_file_create_overwrite.parquet");
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems that we don't need to add a test file. What about using createTempFile() to create a path in your tests here?

@wgtmac
Copy link
Member

wgtmac commented Jun 22, 2023

@amousavigourabi Will you have any update on this?

@amousavigourabi
Copy link
Contributor Author

@amousavigourabi Will you have any update on this?

In crunch mode atm so it took a bit longer, but everything has been addressed now.

Copy link
Member

@wgtmac wgtmac left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM. Thanks!

@wgtmac
Copy link
Member

wgtmac commented Jun 25, 2023

@gszadovszky Do you want to take another pass?

@wgtmac wgtmac merged commit 7c4cb42 into apache:master Jul 4, 2023
9 checks passed
@eyeyar03
Copy link

Hi @wgtmac , just new to exploring and parsing parquet in Java.. I've been trying the sample code here but I can't make it work because of the LocalInputFile not yet in the current version published https://mvnrepository.com/artifact/org.apache.parquet/parquet-common.

This PR seems to be merged to master already though, any reason why I am not seeing the changes in the pushed jar in maven?

Thanks for the help. :)

@wgtmac
Copy link
Member

wgtmac commented Aug 25, 2023

@eyeyar03 We haven't released the next major version 1.14.0 yet, so that's why you cannot see it from there. Usually we don't backport any new feature to a minor release, so the next minor version 1.13.2 will not have it too.

I am not sure what is the best time for the next major release. Could you please advise? @gszadovszky @shangxinli

@gszadovszky
Copy link
Contributor

@wgtmac, in terms of semantic versioning we usually do minor releases (e.g. 1.14.0) for new features and bugfix/patch releases (e.g. 1.13.2) to fix regressions. (A major release would mean 2.0.0 which may contain breaking changes but it is not planned any time soon.)
There is no exact rules for a release. We usually do minor releases if the community feels there are enough features/improvements in master since the last minor release. For bugfix/patch release, we shall do so if a serious regression has been introduced in the last minor releases and we have the fix for it.

@amousavigourabi
Copy link
Contributor Author

@gszadovszky @wgtmac if the next minor release is still far away (1.12.0 and 1.13.0 had over two years between them, with 1.13.0 released a few months ago), I wouldn't mind hosting the two relevant implementations in their own little Maven artifact in the meantime, as there does seem to be some demand.

@gszadovszky
Copy link
Contributor

@amousavigourabi, I would suggest to join the mailing list dev@parquet.apache.org and start a discussion about a potential minor release in the near future.

@eyeyar03
Copy link

eyeyar03 commented Nov 3, 2023

@eyeyar03 We haven't released the next major version 1.14.0 yet, so that's why you cannot see it from there. Usually we don't backport any new feature to a minor release, so the next minor version 1.13.2 will not have it too.

I am not sure what is the best time for the next major release. Could you please advise? @gszadovszky @shangxinli

Thanks @wgtmac . This is noted. Guess we'll have to find an alternative solution for now while waiting for the next major release.

@amousavigourabi amousavigourabi deleted the avoid-hadoop-path branch November 23, 2023 14:06
@drealeed
Copy link

drealeed commented Dec 1, 2023

Our project needs this feature as well, is there a date for the next major release?

@amousavigourabi
Copy link
Contributor Author

Our project needs this feature as well, is there a date for the next major release?

@drealeed if you just need to be able to drop the Hadoop Path dependency, you might want to consider copying the InputFile, OutputFile implementations from this pull request before the next release is out. If you need to fully drop Hadoop, this is still being worked on.

@drealeed
Copy link

drealeed commented Dec 4, 2023

@amousavigourabi , that's actually what I did and it's working for us now. Thanks

@chadselph
Copy link

I'm a big fan of avoiding hadoop dependencies but this change did cause an issue for us. Instantiating the LocalOutputFile creates an outputstream of 128MB, which was causing OOM exceptions for us on our lower-memory deployments. The fix was easy, just set .withRowGroupSize to a smaller value but the old code using hadoop Path did not require this much memory.

@wgtmac
Copy link
Member

wgtmac commented Mar 8, 2024

Hi @chadselph, thanks for reporting the issue. Did you try to debug what's the root cause? Does this relate to compression? This should be unexpected. cc @amousavigourabi

@amousavigourabi
Copy link
Contributor Author

Hi @chadselph, thanks for the report. Like @wgtmac said this should be unexpected. If you could set up an example of where it shows this behaviour that'd be a good start for us to address the issue.

@chadselph
Copy link

chadselph commented Mar 13, 2024

@amousavigourabi @wgtmac sorry for the delay, here's an example

import java.io.IOException;
import java.nio.file.Files;
import org.apache.avro.SchemaBuilder;
import org.apache.avro.generic.GenericData;
import org.apache.parquet.avro.AvroParquetWriter;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
import org.apache.parquet.io.LocalOutputFile;

public class UseMemory {
  public static void main(String[] args) throws IOException {
    var schema =
        SchemaBuilder.builder()
            .record("Record")
            .fields()
            .nullableBoolean("maybe", false)
            .endRecord();
    var memStart = Runtime.getRuntime().freeMemory();
    var temp = Files.createTempFile("parquet-mem", ".parquet");
    Files.delete(temp.toAbsolutePath());
    var writer =
        AvroParquetWriter.<GenericData.Record>builder(new LocalOutputFile(temp.toAbsolutePath()))
            .withCompressionCodec(CompressionCodecName.SNAPPY)
            .withSchema(schema)
            .build();
    System.out.println(
        "Used " + (memStart - Runtime.getRuntime().freeMemory()) / (1024 * 1024) + " MBytes.");
    writer.close();
    Files.delete(temp.toAbsolutePath());
  }
}

I realize this is a crude way to display memory usage, but in case you're in doubt, you can run it in intellij in the debugger and calculate the retained size of the objects, writer is 134MB and all of it is from the output stream created here.

Screenshot 2024-03-13 at 14 49 59

ParquetWriter sets the buffer size to whatever the rowSize is, which defaults to 128MB.

@wgtmac
Copy link
Member

wgtmac commented Mar 14, 2024

Thanks for the info! @chadselph

Do you have time to take an initial look? @amousavigourabi

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
7 participants