-
Notifications
You must be signed in to change notification settings - Fork 2.9k
Build: remove Hadoop 2 dependency #12348
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
Conversation
Now that the minimum java version is 11, it's impossible for Iceberg to work on a Hadoop release less than 3.3.0. Removing the hadoop2 version and libraries forces all building and testing onto a compatible version, and permits followup work using modern hadoop APIs. Co-authored-by: Kristin Cowalcijk <kontinuation@apache.com> Co-authored-by: Steve Loughran <stevel@cloudera.com>
| @Before | ||
| public void resetSpecificConfigurations() { | ||
| spark.conf().unset(COMPRESSION_CODEC); | ||
| spark.conf().unset(COMPRESSION_LEVEL); | ||
| spark.conf().unset(COMPRESSION_STRATEGY); | ||
| } |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
spark.sql.iceberg.compression-level = 1 is not a valid compression level setting for gzip. If we don't unset all these options before running each test, the old configs set by the previous run will be left over and make gzip tests fail.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
for other reviewers: the same is already being done in the Spark 3.5 version of the test and was added by #11333
| md.update(bytes, 0, numBytes); | ||
| } | ||
| return new String(Hex.encodeHex(md.digest())).toUpperCase(Locale.ROOT); | ||
| return Hex.encodeHexString(md.digest(), false); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
why is this changed?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
org.apache.directory.api.util.Hex is not available after switching to Hadoop 3. Also, I think it is more reasonable to use a function from Apache Commons for this purpose.
| // remove the last commit to force Spark to reprocess batch #1 | ||
| File lastCommitFile = new File(checkpoint + "/commits/1"); | ||
| Assert.assertTrue("The commit file must be deleted", lastCommitFile.delete()); | ||
| Files.deleteIfExists(Paths.get(checkpoint + "/commits/.1.crc")); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
why do we need this now?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The .crc file will be renamed along with the main file since HADOOP-16255, deleting the main file without deleting the crc file will result in a failure when renaming to the main file again:
org.apache.hadoop.fs.FileAlreadyExistsException: Rename destination file:/var/folders/jw/nz45tb550rbgjkndd37m8rrh0000gn/T/junit-12664551068658781194/parquet/checkpoint/commits/.1.crc already exists.
at org.apache.hadoop.fs.AbstractFileSystem.renameInternal(AbstractFileSystem.java:876)
at org.apache.hadoop.fs.AbstractFileSystem.rename(AbstractFileSystem.java:807)
at org.apache.hadoop.fs.ChecksumFs.renameInternal(ChecksumFs.java:519)
at org.apache.hadoop.fs.AbstractFileSystem.rename(AbstractFileSystem.java:807)
at org.apache.hadoop.fs.FileContext.rename(FileContext.java:1044)
at org.apache.spark.sql.execution.streaming.FileContextBasedCheckpointFileManager.renameTempFile(CheckpointFileManager.scala:372)
at org.apache.spark.sql.execution.streaming.CheckpointFileManager$RenameBasedFSDataOutputStream.close(CheckpointFileManager.scala:154)
at org.apache.spark.sql.execution.streaming.HDFSMetadataLog.write(HDFSMetadataLog.scala:204)
at org.apache.spark.sql.execution.streaming.HDFSMetadataLog.addNewBatchByStream(HDFSMetadataLog.scala:237)
at org.apache.spark.sql.execution.streaming.HDFSMetadataLog.add(HDFSMetadataLog.scala:130)
at org.apache.spark.sql.execution.streaming.MicroBatchExecution.$anonfun$markMicroBatchEnd$1(MicroBatchExecution.scala:785)
|
The title sounds a bit strange to me. How about |
Renamed the title as requested. |
|
Looks good, thanks @Kontinuation for working on this, and thanks @manuzhang and @nastra for the review! |
This is the continuation of #10932
The removal of Hadoop 2 was blocked by hive2 before. Now we have hive runtime removed from the source tree since #11801, Hadoop 2 removal should have been unblocked.
Removing Hadoop 2 is also required for upgrading parquet packages to the next version, as described in #12347 (comment), latest parquet-hadoop code uses new FileSystem APIs introduced in Hadoop 3.
Related issue: #10940