Skip to content

Conversation

@dlvenable
Copy link
Contributor

@dlvenable dlvenable commented Jun 18, 2024

If you want to write an unencrypted Parquet file without Hadoop, the existing code will use Hadoop to try to get encryption properties.

if (encryptionProperties == null) {
String path = file == null ? null : file.getPath();
Configuration hadoopConf = ConfigurationUtil.createHadoopConfiguration(conf);
encryptionProperties = ParquetOutputFormat.createEncryptionProperties(
hadoopConf, path == null ? null : new Path(path), writeContext);
}

However, if you have these null, we really didn't need to go through Hadoop. Also, it calls a helper method in ParquetOutputFormat. This class inherits from Hadoop's FileOutputFormat. So calling this method at all, requires Hadoop classes. To resolve this, I moved this helper into a package-protected EncryptionPropertiesHelper class.

Closes #1497

… Hadoop in the classpath. Relates to PARQUET-1126.
@wgtmac wgtmac changed the title Write unencrypted Parquet files without Hadoop PARQUET-1126 PARQUET-1126: Write unencrypted Parquet files without Hadoop Jun 19, 2024

Configuration hadoopConf = ConfigurationUtil.createHadoopConfiguration(fileParquetConfig);
URI path = tempFilePath == null ? null : tempFilePath.toUri();
return createEncryptionProperties(
Copy link
Contributor

Choose a reason for hiding this comment

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

this will trigger a second loading of the factory class. Can we replace ll40-41 with the l52 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.

@ggershinsky , Thank you for taking a look at the PR. This is a good point. I corrected it in the latest commit.

@dlvenable
Copy link
Contributor Author

@ggershinsky , Thank you for the approval. Is anything else necessary to merge this?

@wgtmac wgtmac merged commit 7be9807 into apache:master Jul 30, 2024
@wgtmac
Copy link
Member

wgtmac commented Jul 30, 2024

I just merged this. Thanks @dlvenable and @ggershinsky!

@Fokko Fokko added this to the 1.14.2 milestone Aug 15, 2024
Fokko pushed a commit that referenced this pull request Aug 15, 2024
* Internal changes to allow writing unencrypted Parquet without needing Hadoop in the classpath. Relates to PARQUET-1126.
* Avoid loading encryption factory twice.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

make it easy to read and write parquet files in java without depending on hadoop

4 participants