Skip to content

Support Java 25#5883

Open
clairemcginty wants to merge 7 commits intomainfrom
run-java-25
Open

Support Java 25#5883
clairemcginty wants to merge 7 commits intomainfrom
run-java-25

Conversation

@clairemcginty
Copy link
Contributor

@clairemcginty clairemcginty commented Feb 10, 2026

Java 25 is supported as of Beam 2.69. Let's support in Scio too

Change summary for Java 25 upgrade:

  • CI runtimes shifted from [11, 17, 21] to [17, 21, 25]. Note that we're still compiling to 11 bytecode.
  • Fix Parquet writes by removing usages of Hadoop's org.apache.hadoop.mapreduce.Job API, which we used as a convenient shorthand for managing distinct Configurations, and just using org.apache.hadoop.conf.Configuration objects directly. (See comment below)
  • Fix Parquet "legacy", or non-SplittableDoFn, reads, for both .parquetAvroFile[T](...) and .parquetExampleFile, but not for .typedParquetFile, due to a limitation with the current magnolify-parquet implementation preventing us from totally tearing out uses of the Job API. However, SplittableDoFn-based reads (which are currently enabled by default) work across all Parquet read formats.
  • Fix AutoValue generation by adding the -proc:full java option (thanks Claude!)
  • Fix parsing of --optionsFile=... by explicitly registering ScioOptions

Testing:

)(ScioUtil.strippedPath(path), suffix)
val dynamicDestinations = DynamicFileDestinations
.constant(fp, SerializableFunctions.identity[T])
val job = Job.getInstance(conf)
Copy link
Contributor Author

Choose a reason for hiding this comment

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

The Job API has known issues with Java 25; it triggers the Subject#getSubject call discussed here: apache/parquet-java#3328

We don't actually need to use Job; we were just using it as a convenient way to manage distinct Configurations, but I don't see a reason we can't drop Job and just use Configuration directly.

Copy link
Contributor

Choose a reason for hiding this comment

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

Calling Job.getInstance(conf) seems to copy the conf values into a new object while this would mutate the input conf ... could be an issue in e.g. tests or something?

@codecov
Copy link

codecov bot commented Feb 10, 2026

Codecov Report

❌ Patch coverage is 40.54054% with 22 lines in your changes missing coverage. Please review.
✅ Project coverage is 61.54%. Comparing base (fb98d33) to head (2fcec27).
⚠️ Report is 1 commits behind head on main.

Files with missing lines Patch % Lines
...ify/scio/parquet/tensorflow/ParquetExampleIO.scala 18.18% 9 Missing ⚠️
...la/com/spotify/scio/parquet/GcsConnectorUtil.scala 36.36% 7 Missing ⚠️
...com/spotify/scio/parquet/types/ParquetTypeIO.scala 25.00% 6 Missing ⚠️
Additional details and impacted files
@@            Coverage Diff             @@
##             main    #5883      +/-   ##
==========================================
+ Coverage   61.52%   61.54%   +0.01%     
==========================================
  Files         317      317              
  Lines       11663    11653      -10     
  Branches      869      822      -47     
==========================================
- Hits         7176     7172       -4     
+ Misses       4487     4481       -6     

☔ View full report in Codecov by Sentry.
📢 Have feedback on the report? Share it here.

🚀 New features to boost your workflow:
  • ❄️ Test Analytics: Detect flaky tests, report on failures, and find test suite problems.

The test "support options from optionsFile" was failing on Java 25 because
ScioOptions was not explicitly registered with PipelineOptionsFactory,
causing getOptionsFile to return null.

Changes:
- Register ScioOptions in parseArguments to ensure --optionsFile is recognized

Co-Authored-By: Claude Sonnet 4.5 <noreply@anthropic.com>
@@ -90,7 +90,7 @@ final case class ParquetTypeIO[T: ClassTag: Coder: ParquetType](
val cls = ScioUtil.classOf[T]
val job = Job.getInstance(conf)
Copy link
Contributor Author

Choose a reason for hiding this comment

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

I was unable to remove this instance of Job API becuase Magnolify's ParquetType#setupInput requires it: https://github.com/spotify/magnolify/blob/v0.9.3/parquet/src/main/scala/magnolify/parquet/ParquetType.scala#L66-L70

It references some private methods so I can't just inline the method logic. We can refactor this in a future Magnolify version.

Thus, legacy (non-SDF) typed Parquet reads are currently not supported on Java 25. However, based on the linked parquet-java thread, looks like the issue with the Job API is set to be fixed in the upcoming Hadoop 3.4.3 release (unfortunately the 3.4.3 release candidate resolver link expired so I wasn't able to validate that).

Copy link
Contributor

Choose a reason for hiding this comment

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

This is a stale comment after your magnolify PR, yes? We should just do a magnolify release to support both variants and remove the Job dep here

@clairemcginty clairemcginty changed the title (wip) Support Java 25 Support Java 25 Feb 10, 2026
@clairemcginty clairemcginty marked this pull request as ready for review February 10, 2026 20:18
): (T, Args) = {
val optClass = ScioUtil.classOf[T]
PipelineOptionsFactory.register(optClass)
PipelineOptionsFactory.register(classOf[ScioOptions])
Copy link
Contributor

Choose a reason for hiding this comment

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

🤔 was this a bug?

)(ScioUtil.strippedPath(path), suffix)
val dynamicDestinations = DynamicFileDestinations
.constant(fp, SerializableFunctions.identity[T])
val job = Job.getInstance(conf)
Copy link
Contributor

Choose a reason for hiding this comment

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

Calling Job.getInstance(conf) seems to copy the conf values into a new object while this would mutate the input conf ... could be an issue in e.g. tests or something?

@@ -90,7 +90,7 @@ final case class ParquetTypeIO[T: ClassTag: Coder: ParquetType](
val cls = ScioUtil.classOf[T]
val job = Job.getInstance(conf)
Copy link
Contributor

Choose a reason for hiding this comment

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

This is a stale comment after your magnolify PR, yes? We should just do a magnolify release to support both variants and remove the Job dep here


FileInputFormat.setInputPaths(job, path)
GcsConnectorUtil.setCredentials(conf)
conf.set(FileInputFormat.INPUT_DIR, StringUtils.escapeString(path));
Copy link
Contributor

Choose a reason for hiding this comment

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

If we're not calling FileInputFormat.setInputPaths then the creds aren't required, no?

Also the internals of setInputPaths is more like:

val p = StringUtils.stringToPath(path)
val inputDir = StringUtils.escapeString(p.getFileSystem(conf).makeQualified(p).toString());
conf.set(FileInputFormat.INPUT_DIR, inputDir);

Copy link
Contributor Author

Choose a reason for hiding this comment

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

yeah I saw that - but I don't think it's needed for our use case where we know path is either a single file path or glob. Beam already checks filesystem prefix no?

Copy link
Contributor

Choose a reason for hiding this comment

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

I thought the qualification part might differ between filesystems but tbh I didn't dig into that level

val rows = Seq(
(() => createConfig(true), "splittable"),
(() => ParquetAvroIO.ReadParam.DefaultConfiguration, "default")
) ++ (
Copy link
Contributor

Choose a reason for hiding this comment

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

Bump magnolify post-release and this should be OK, no?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Sadly no - that issue is solved and a new one GetSubject call popped up coming from Beam's HadoopFormatIO internals

(the other Parquet[Type,Example]IO tests don't test real reads on the legacy pathway, so we only need it here...)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Validated that latest Hadoop 3.4.3 RC fixes the outstanding issues with parquet legacy reads. not sure when the planned stable release date is though.

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.

2 participants