Skip to content

Conversation

@stoty
Copy link
Contributor

@stoty stoty commented Aug 22, 2025

Description of PR

HADOOP-19574. Restore Subject propagation semantics for Java 22+.

JDK 22 breaks subject propagation into new Threads.
This patch adds a new HadoopThread class which restores the pre JDK22 semantics, and replaces
most Thread objects in the code with HadoopThread.
The Daemon class is silimarly changed.

This is the -hopefully- final patch for full Java 24/25 support.

I went with the approach of replacing almost all Thread objects with HadoopThread, as most of the tests do not set any specific Principal, and it would be had to determine for every Thread usage whether Subject propagation is required for that instance.

While the patch is huge, 99% of it is just mechanically replacing Thread with HadoopThread, and the run() method with the work() method.
There are a few tests which check specific stack traces, which also needed updating.

I'm open to renaming anything if someone can come up with better names.

How was this patch tested?

./start-build-env.sh
export JAVA_HOME=/usr/lib/jvm/temurin-24-jdk-amd64/
mvn clean test -fn 

There are a few failing tests, but those are either flakey or also fail on trunk and / or with Java 8 / 17.

For code changes:

  • Does the title or this PR starts with the corresponding JIRA issue id (e.g. 'HADOOP-17799. Your PR title ...')?
  • Object storage: have the integration tests been executed and the endpoint declared according to the connector-specific documentation?
  • If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under ASF 2.0?
  • If applicable, have you updated the LICENSE, LICENSE-binary, NOTICE-binary files?

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 0m 0s Docker mode activated.
-1 ❌ patch 0m 21s #7892 does not apply to trunk. Rebase required? Wrong Branch? See https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute for help.
Subsystem Report/Notes
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7892/1/console
versions git=2.34.1
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 0m 0s Docker mode activated.
-1 ❌ patch 0m 22s #7892 does not apply to trunk. Rebase required? Wrong Branch? See https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute for help.
Subsystem Report/Notes
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7892/2/console
versions git=2.34.1
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 0m 0s Docker mode activated.
-1 ❌ patch 0m 21s #7892 does not apply to trunk. Rebase required? Wrong Branch? See https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute for help.
Subsystem Report/Notes
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7892/3/console
versions git=2.34.1
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 0m 0s Docker mode activated.
-1 ❌ patch 0m 21s #7892 does not apply to trunk. Rebase required? Wrong Branch? See https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute for help.
Subsystem Report/Notes
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7892/4/console
versions git=2.34.1
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 0m 0s Docker mode activated.
-1 ❌ patch 0m 21s #7892 does not apply to trunk. Rebase required? Wrong Branch? See https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute for help.
Subsystem Report/Notes
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7892/5/console
versions git=2.34.1
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 0m 0s Docker mode activated.
-1 ❌ patch 0m 21s #7892 does not apply to trunk. Rebase required? Wrong Branch? See https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute for help.
Subsystem Report/Notes
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7892/6/console
versions git=2.34.1
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 0m 0s Docker mode activated.
-1 ❌ patch 0m 21s #7892 does not apply to trunk. Rebase required? Wrong Branch? See https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute for help.
Subsystem Report/Notes
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7892/7/console
versions git=2.34.1
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 0m 0s Docker mode activated.
-1 ❌ patch 0m 32s #7892 does not apply to trunk. Rebase required? Wrong Branch? See https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute for help.
Subsystem Report/Notes
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7892/8/console
versions git=2.34.1
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

@stoty stoty marked this pull request as ready for review August 27, 2025 12:11
@stoty
Copy link
Contributor Author

stoty commented Aug 27, 2025

This is final piece for Java 24 / 25 support.
While the patch is huge, the actual changes are very simple.
HadoopThread was added and Daemon was modified, the rest is just simply replacement.

Please take a look @pan3793 @slfan1989 @jojochuang @cnauroth @steveloughran

@slfan1989
Copy link
Contributor

This is final piece for Java 24 / 25 support. While the patch is huge, the actual changes are very simple. HadoopThread was added and Daemon was modified, the rest is just simply replacement.

Please take a look @pan3793 @slfan1989 @jojochuang @cnauroth @steveloughran

I have a few experienced members to recommend for the review.

@Hexiaoqiao @ayushtkn @szetszwo I need your help to collaboratively review this PR. Thank you very much!

Copy link
Contributor

@cnauroth cnauroth left a comment

Choose a reason for hiding this comment

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

Hi @stoty . It looks like Yetus is saying this won't apply to trunk. Does it need a rebase? I also just committed MAPREDUCE-7502, so if you rebase, the test fix won't need to be applied manually anymore.

@stoty
Copy link
Contributor Author

stoty commented Aug 27, 2025

Unfortunately Yetus cannot handle patches this big, I don't think we can get CI run via GitHub.

@szetszwo
Copy link
Contributor

szetszwo commented Sep 2, 2025

You cannot override the run() / work() method if you use a factory.

@stoty , builder is more flexible than constructors. We definitely can override method as today.

//TestByteArrayManager
  static class AllocatorThread extends Thread {
// existing code
        final AllocatorThread t = new AllocatorThread(arrayLength, bam);

Instead of call new AllocatorThread(..) above, we will have

// new code
        final Thread t =  SubjectInheritingThread.newBuilder()
            .setThread(new AllocatorThread(arrayLength, bam))
            .build();

@stoty
Copy link
Contributor Author

stoty commented Sep 3, 2025

I spent some time trying to implement that @szetszwo .

At the core, we'd need a wrapper like this:

`
public static class SubjectInheritingWrapper extends Thread {

private Thread wrapped;
private Subject startSubject;

public SubjectInheritingWrapper(Thread wrapped) {
  this.wrapped = wrapped;
}

@Override
public synchronized void start() {
  startSubject = SubjectUtil.current();
  wrapped.start();
}

@Override
public final void run() {
  SubjectUtil.doAs(startSubject, new PrivilegedAction<Void>() {

    @Override
    public Void run() {
      wrapped.run();
      return null;
    }

  });
}

public void interrupt() {
  wrapped.interrupt();
}

public boolean isInterrupted() {
  return wrapped.isInterrupted();
}

// setDaemon is final!!!
public void setDaemon()...

}`

run() and start() are wrappable, but Thread is full of final methods which cannot be overriden, and would not work on the wrapper (which is never actually started, only the wrapped thread is).

Say, we want to call setDaemon() on the wrapper. Since we cannot override it, it would set the daemon flag on the wrapper, not the wrapped thread, which is never even start() ed , and wouldn't work.

So a wrapper like this would invisibly break the Thread contract in many ways.
I don't think it's worth the risk of the hidden bugs that this could cause.

@szetszwo
Copy link
Contributor

szetszwo commented Sep 3, 2025

@stoty , thanks for trying it out! The code should look like below:

public class SubjectDoAsThread extends Thread {
  private final Runnable runnable;
  private Subject startSubject;

  public static class Builder {
    private ThreadGroup group;
    private String name;
    private Runnable runnable;

    public Builder setThread(Thread thread) {
      this.group = thread.getThreadGroup();
      this.name = thread.getName();
      this.runnable = thread;
      return this;
    }

    public SubjectDoAsThread build() {
      return new SubjectDoAsThread(group, name, runnable);
    }
  }

  private SubjectDoAsThread(ThreadGroup group, String name, Runnable runnable) {
    super(group, name);
    this.runnable = Objects.requireNonNull(runnable, "runnable == null");
  }

  @Override
  public final void start() {
    startSubject = SubjectUtil.current();
    super.start();
  }

  @Override
  public final void run() {
    SubjectUtil.doAs(startSubject, (PrivilegedAction<Void>) () -> {
      runnable.run();
      return null;
    });
  }
}

@stoty
Copy link
Contributor Author

stoty commented Sep 4, 2025

That looks like it could work @szetszwo .

There is still the issue that if there is any logic in the wrapped Thread's start() method, then it won't be run,
but we can document that (and the current SubjectInheritingThread also makes that final, so we know that none of the Threads in Hadoop does that)

@stoty
Copy link
Contributor Author

stoty commented Sep 9, 2025

I've spent some more tim on this @szetszwo , and I am even more convinced that using the Builder approach is NOT a good universal solution.

I've implemented something close to your example above, then I randomy chose org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer to test it.

Two problems became immediately apparent with the builder approach:

  1. EditLogTailerThread has an internal state and a setShouldRun() method to set it. This immediately makes the Builder approach you've shown a non-starter, as the thread created by the Builder is of a different Type, and does not have a setShouldRun() method (nor the required internal state)

  2. An EditLogTailerThread member is defined in the class. Even if we could get away with changing the type (which in many cases we can do), we'd still have to either
    a.) change the type to Thread, thereby weakening the type checks
    b.) add another member for the wrapped thread, in which case we need go over the code, and carefully check which member we need to use where, adding a lot of possibilities for mistakes.

That means that only the Threads which set a Runner can be converted easily, the ones which subclass Thread would need to be evaulated on a case-by-case basis, which is exactly what I don't want to do due to both the time requirement and the error-prone nature of the process.

@pan3793 @slfan1989 @jojochuang @cnauroth @steveloughran @Hexiaoqiao @ayushtkn @szetszwo

Please review the current state of #7919 which is the same patch as this split into three and with some minor improvements.

My plan is to rebase/reorder the #7919 . Please check #7919

  • Do you think we should implement and use Thread Builder approach in this patch ? (see my reservations above)
  • Is the way I split the patch acceptable (patches without JIRA id will be merged into the other ones)
  • What other changes (if any) do you think are necessary for patch to be accepted M

@szetszwo
Copy link
Contributor

szetszwo commented Sep 9, 2025

@stoty , the code will look like below:

diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
index cf416307f47..3260b770ac0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
@@ -85,8 +85,8 @@ public class EditLogTailer {
   public static final long DFS_HA_TAILEDITS_MAX_TXNS_PER_LOCK_DEFAULT =
       Long.MAX_VALUE;
 
-  private final EditLogTailerThread tailerThread;
-  
+  private final SubjectDoAsThread<EditLogTailerThread> tailerThread;
+
   private final Configuration conf;
   private final FSNamesystem namesystem;
   private final Iterator<RemoteNameNodeInfo> nnLookup;
@@ -180,7 +180,11 @@ public class EditLogTailer {
   private Timer timer;
 
   public EditLogTailer(FSNamesystem namesystem, Configuration conf) {
-    this.tailerThread = new EditLogTailerThread();
+    final EditLogTailerThread t = new EditLogTailerThread();
+    this.tailerThread = SubjectDoAsThread.<EditLogTailerThread>newBuilder()
+        .setThread(new EditLogTailerThread())
+        .setOriginal(t)
+        .build();
     this.conf = conf;
     this.namesystem = namesystem;
     this.timer = new Timer();
@@ -271,7 +275,7 @@ public void start() {
   }
   
   public void stop() throws IOException {
-    tailerThread.setShouldRun(false);
+    tailerThread.getOriginal().setShouldRun(false);
     tailerThread.interrupt();
     try {
       tailerThread.join();
@@ -664,6 +668,6 @@ public RemoteNameNodeInfo getCurrentNN() {
 
   @VisibleForTesting
   public void setShouldRunForTest(boolean shouldRun) {
-    this.tailerThread.setShouldRun(shouldRun);
+    this.tailerThread.getOriginal().setShouldRun(shouldRun);
   }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/SubjectDoAsThread.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/SubjectDoAsThread.java
new file mode 100644
index 00000000000..edac81048a6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/SubjectDoAsThread.java
@@ -0,0 +1,64 @@
+package org.apache.hadoop.hdfs.server.namenode.ha;
+
+import org.apache.hadoop.security.authentication.util.SubjectUtil;
+
+import javax.security.auth.Subject;
+import java.security.PrivilegedAction;
+import java.util.Objects;
+
+public class SubjectDoAsThread<T> extends Thread {
+  private final T original;
+  private final Runnable runnable;
+  private Subject startSubject;
+
+  public static class Builder<T> {
+    private ThreadGroup group;
+    private String name;
+    private Runnable runnable;
+    private T original;
+
+    public Builder<T> setThread(Thread thread) {
+      this.group = thread.getThreadGroup();
+      this.name = thread.getName();
+      this.runnable = thread;
+      return this;
+    }
+
+    public Builder<T> setOriginal(T original) {
+      this.original = original;
+      return this;
+    }
+
+    public SubjectDoAsThread<T> build() {
+      return new SubjectDoAsThread<>(group, name, original, runnable);
+    }
+  }
+
+  private SubjectDoAsThread(ThreadGroup group, String name, T original, Runnable runnable) {
+    super(group, name);
+    this.original = original;
+    this.runnable = Objects.requireNonNull(runnable, "runnable == null");
+  }
+
+  static <T> Builder<T> newBuilder() {
+    return new Builder<>();
+  }
+
+  @Override
+  public final void start() {
+    startSubject = SubjectUtil.current();
+    super.start();
+  }
+
+  @Override
+  public final void run() {
+    SubjectUtil.doAs(startSubject, (PrivilegedAction<Void>) () -> {
+      runnable.run();
+      return null;
+    });
+  }
+
+  public T getOriginal() {
+    return original;
+  }
+}

@slfan1989
Copy link
Contributor

@stoty Thank you for your contribution! However, I may not be able to thoroughly review this PR until next Monday or Tuesday.

Currently, the JUnit 5 upgrade for the HDFS module has been mostly completed. I'm now working on completely removing the dependency on JUnit 4 from the trunk branch of the Hadoop project. This may take some time, as there are still a few missing pieces that need to be fixed.

@pan3793
Copy link
Member

pan3793 commented Sep 10, 2025

Do you think we should implement and use Thread Builder approach in this patch ? (see my reservations above)

I think introducing Builder here is a kind of overdesign. @szetszwo do you have cases in your mind that can not be covered by @stoty's approach? If not, I strongly support @stoty's approach.

Is the way I split the patch acceptable (patches without JIRA id will be merged into the other ones)

Thanks for splitting! I think HADOOP-19668 and HADOOP-19669 can be committed together; the current state is also fine.

What other changes (if any) do you think are necessary for patch to be accepted?

I think it's already in good shape.

@stoty
Copy link
Contributor Author

stoty commented Sep 10, 2025

Yes, the solution you propose works @szetszwo , but it's more complex and error-prone ot implement than the current one.

Now we are juggling two thread objects, and it is very easy to start the wrong one, especially when you're converting hundreds of threads.

@slfan1989
Copy link
Contributor

@stoty I’d like to confirm with you whether you’d prefer us to create a separate development branch for you.

Pan and I had an offline discussion on this topic. He believes the changes are manageable and therefore a separate branch may not be necessary. I think his point makes sense to some extent, but from the perspective of development efficiency and risk control, I would still recommend creating a dedicated development branch. This would allow us to iterate more quickly without impacting existing functionality.

@stoty
Copy link
Contributor Author

stoty commented Sep 12, 2025

@stoty I’d like to confirm with you whether you’d prefer us to create a separate development branch for you.

Pan and I had an offline discussion on this topic. He believes the changes are manageable and therefore a separate branch may not be necessary. I think his point makes sense to some extent, but from the perspective of development efficiency and risk control, I would still recommend creating a dedicated development branch. This would allow us to iterate more quickly without impacting existing functionality.

I don't think a separate branch is necessary, @slfan1989 .

These changes are designed to be transparent, especially now that we've added a shortcut to avoid the subject manipulation for Java 21 and earlier, as suggested by @szetszwo .

IF this breaks anything, we can find and fix that faster once it's in trunk.

Any changes affecting Java 8/11 will be causght by the existing CI processes, and we can also start work on setting up CI for Java 17/21/25 and start working out any such issues.

Users will alse have an easier time testing Hadoop with Java 21/25 in their use cases once it's supported on trunk.

@stoty
Copy link
Contributor Author

stoty commented Sep 12, 2025

A related question @slfan1989 :
What is current stance on dropping pre Java 17 support from trunk ?
That would allow freeing up CI resources, and start migrating off problematic old component versions like Jetty 9.4.

@slfan1989
Copy link
Contributor

@stoty I’d like to confirm with you whether you’d prefer us to create a separate development branch for you.
Pan and I had an offline discussion on this topic. He believes the changes are manageable and therefore a separate branch may not be necessary. I think his point makes sense to some extent, but from the perspective of development efficiency and risk control, I would still recommend creating a dedicated development branch. This would allow us to iterate more quickly without impacting existing functionality.

I don't think a separate branch is necessary, @slfan1989 .

These changes are designed to be transparent, especially now that we've added a shortcut to avoid the subject manipulation for Java 21 and earlier, as suggested by @szetszwo .

IF this breaks anything, we can find and fix that faster once it's in trunk.

Any changes affecting Java 8/11 will be causght by the existing CI processes, and we can also start work on setting up CI for Java 17/21/25 and start working out any such issues.

Users will alse have an easier time testing Hadoop with Java 21/25 in their use cases once it's supported on trunk.

@stoty Thank you for your reply, and also for your efforts in driving the Java 22+ upgrade. I hope the work goes smoothly, and I'll do my best to support it.

A related question @slfan1989 :
What is current stance on dropping pre Java 17 support from trunk ?
That would allow freeing up CI resources, and start migrating off problematic old component versions like Jetty 9.4.

From my understanding, the trunk branch will soon drop support for Java 8 and Java 11, and move forward with Java 17 as the new baseline for development. We also plan to finalize the release scope of Hadoop 3.5 as soon as possible and initiate the corresponding release process.

As for CI support for Java 21 and above, further discussion may be needed. I plan to respond to the previous DISCUSS thread early next week and continue driving the related efforts. In the meantime, input and suggestions from other community members would be very welcome.

@pan3793
Copy link
Member

pan3793 commented Sep 15, 2025

Kindly ping @szetszwo @steveloughran @cnauroth

We need a final decision here. I think these two approaches are just different programming patterns, with no essential functionality difference. Java 25 is coming (scheduled on Sept 26, 2025), and I am eager for a Java 25-compatible Hadoop client to unblock downstream projects to support Java 25.

@szetszwo
Copy link
Contributor

@stoty , @pan3793 , Honestly, I do prefer adding a builder instead of having multiple constructors with different parameter combinations. Also, a builder can build subclasses but constructors can't.

In addition, extending the Thread class is not a good practice -- it is better to implement Runnable; see https://stackoverflow.com/questions/541487/implements-runnable-vs-extends-thread-in-java . We are a kind of making the ugly code worse.

However, I won't -1 if we are not adding builder. It is just a suggestion for your consideration. I am fine if other people have agreed to do it differently.

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 0m 0s Docker mode activated.
-1 ❌ patch 0m 24s #7892 does not apply to trunk. Rebase required? Wrong Branch? See https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute for help.
Subsystem Report/Notes
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7892/14/console
versions git=2.34.1
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

@stoty
Copy link
Contributor Author

stoty commented Sep 22, 2025

I have overwritten this PR with the split one, and rebased it to current trunk.
This is now the same as #7919 but without the Yetus changes for test enablement.

Copy link
Contributor

@steveloughran steveloughran left a comment

Choose a reason for hiding this comment

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

wow, this is a big patch. but we need it, don't we?

only looked at it briefly -will need to go through the full change before approval but I don't see any reason to block the change so far

}
}

private static boolean checkThreadInheritsSubject() {
Copy link
Contributor

Choose a reason for hiding this comment

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

nit, javadoc

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Added javadoc

final String name =
prefix + "-pool" + poolNum + "-t" + threadNumber.getAndIncrement();
return new Thread(group, r, name);
return new SubjectInheritingThread(group, r, name);
Copy link
Contributor

Choose a reason for hiding this comment

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

nice. I was wondering about this

import java.util.concurrent.atomic.AtomicBoolean;

import org.apache.hadoop.classification.VisibleForTesting;
import org.apache.hadoop.util.concurrent.SubjectInheritingThread;
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: import ordering

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fixed

* This is the equivalent of {@link Thread#run()}. Override this instead of
* {@link #run()} Subject will be propagated like in pre-Java 22 Thread.
*/
public void work() {
Copy link
Contributor

Choose a reason for hiding this comment

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

PITA having to rename all uses, but I can't see any other workaround

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 only way would be converting everything to use a separate Runnable instead of subclassing Thread as suggested by Nicholas, but IMO that would be a much more intrusive change, with a high chance of getting some the hundreds of conversions wrong.

@stoty
Copy link
Contributor Author

stoty commented Oct 2, 2025

wow, this is a big patch. but we need it, don't we?

Yes. IMO the Java language development process has really dropped the ball here.
Changing the Subject APIs while providing replacements is one thing, but breaking the whole Java authorization model was a big mistake.

Strictly speaking we don't need to convert EVERY Thread instance, especially in the non-secure tests, but figuring out the cases where we can safely drop the Subject / UGI from the new Thread would be super hard and error-prone (and also fragile in case authenticated operations are added later there).

only looked at it briefly -will need to go through the full change before approval but I don't see any reason to block the change so far

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 22m 57s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 0s No case conflicting files found.
+0 🆗 codespell 0m 1s codespell was not available.
+0 🆗 detsecrets 0m 1s detect-secrets was not available.
-1 ❌ @author 0m 0s /results-author.txt The patch appears to contain 4 @author tags which the community has agreed to not allow in code contributions.
-1 ❌ test4tests 0m 0s The patch doesn't appear to include any new or modified tests. Please justify why no new tests are needed for this patch. Also please list what manual steps were performed to verify this patch.
_ trunk Compile Tests _
-1 ❌ mvninstall 53m 0s /branch-mvninstall-root.txt root in trunk failed.
+1 💚 compile 1m 28s trunk passed with JDK Ubuntu-11.0.27+6-post-Ubuntu-0ubuntu120.04
+1 💚 compile 1m 14s trunk passed with JDK Private Build-1.8.0_452-8u452-gaus1-0ubuntu120.04-b09
+1 💚 checkstyle 0m 55s trunk passed
+1 💚 mvnsite 1m 22s trunk passed
+1 💚 javadoc 1m 11s trunk passed with JDK Ubuntu-11.0.27+6-post-Ubuntu-0ubuntu120.04
+1 💚 javadoc 1m 41s trunk passed with JDK Private Build-1.8.0_452-8u452-gaus1-0ubuntu120.04-b09
+1 💚 spotbugs 3m 12s trunk passed
+1 💚 shadedclient 42m 11s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
-1 ❌ mvninstall 0m 30s /patch-mvninstall-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch failed.
-1 ❌ compile 0m 31s /patch-compile-hadoop-hdfs-project_hadoop-hdfs-jdkUbuntu-11.0.27+6-post-Ubuntu-0ubuntu120.04.txt hadoop-hdfs in the patch failed with JDK Ubuntu-11.0.27+6-post-Ubuntu-0ubuntu120.04.
-1 ❌ javac 0m 31s /patch-compile-hadoop-hdfs-project_hadoop-hdfs-jdkUbuntu-11.0.27+6-post-Ubuntu-0ubuntu120.04.txt hadoop-hdfs in the patch failed with JDK Ubuntu-11.0.27+6-post-Ubuntu-0ubuntu120.04.
-1 ❌ compile 0m 29s /patch-compile-hadoop-hdfs-project_hadoop-hdfs-jdkPrivateBuild-1.8.0_452-8u452-gaus1-0ubuntu120.04-b09.txt hadoop-hdfs in the patch failed with JDK Private Build-1.8.0_452-8u452-gaus1-0ubuntu120.04-b09.
-1 ❌ javac 0m 29s /patch-compile-hadoop-hdfs-project_hadoop-hdfs-jdkPrivateBuild-1.8.0_452-8u452-gaus1-0ubuntu120.04-b09.txt hadoop-hdfs in the patch failed with JDK Private Build-1.8.0_452-8u452-gaus1-0ubuntu120.04-b09.
+1 💚 blanks 0m 0s The patch has no blanks issues.
-0 ⚠️ checkstyle 0m 18s /buildtool-patch-checkstyle-hadoop-hdfs-project_hadoop-hdfs.txt The patch fails to run checkstyle in hadoop-hdfs
-1 ❌ mvnsite 0m 30s /patch-mvnsite-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch failed.
-1 ❌ javadoc 0m 31s /patch-javadoc-hadoop-hdfs-project_hadoop-hdfs-jdkUbuntu-11.0.27+6-post-Ubuntu-0ubuntu120.04.txt hadoop-hdfs in the patch failed with JDK Ubuntu-11.0.27+6-post-Ubuntu-0ubuntu120.04.
-1 ❌ javadoc 0m 30s /patch-javadoc-hadoop-hdfs-project_hadoop-hdfs-jdkPrivateBuild-1.8.0_452-8u452-gaus1-0ubuntu120.04-b09.txt hadoop-hdfs in the patch failed with JDK Private Build-1.8.0_452-8u452-gaus1-0ubuntu120.04-b09.
-1 ❌ spotbugs 0m 29s /patch-spotbugs-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch failed.
-1 ❌ shadedclient 11m 13s patch has errors when building and testing our client artifacts.
_ Other Tests _
-1 ❌ unit 0m 32s /patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch failed.
-1 ❌ asflicense 0m 33s /results-asflicense.txt The patch generated 1 ASF License warnings.
142m 39s
Subsystem Report/Notes
Docker ClientAPI=1.51 ServerAPI=1.51 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7892/15/artifact/out/Dockerfile
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets
uname Linux fbae4de0f31e 5.15.0-156-generic #166-Ubuntu SMP Sat Aug 9 00:02:46 UTC 2025 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / d115595
Default Java Private Build-1.8.0_452-8u452-gaus1-0ubuntu120.04-b09
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.27+6-post-Ubuntu-0ubuntu120.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_452-8u452-gaus1-0ubuntu120.04-b09
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7892/15/testReport/
Max. process+thread count 529 (vs. ulimit of 5500)
modules C: hadoop-hdfs-project/hadoop-hdfs U: hadoop-hdfs-project/hadoop-hdfs
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7892/15/console
versions git=2.25.1 maven=3.9.11 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

@pan3793
Copy link
Member

pan3793 commented Oct 15, 2025

Kindly ping @steveloughran, do you have further comments/advisors on this PR. There has been about one month since Java 25 GA, I see an increasing number of big data projects that depend on hadoop blocked by this PR to support Java 25.

@pan3793
Copy link
Member

pan3793 commented Nov 3, 2025

I'm willing to help if there is anything I can do to move this forward

cc more Apache Hadoop PMC members @szetszwo @Hexiaoqiao @cnauroth @sunchao @steveloughran @slfan1989

@steveloughran
Copy link
Contributor

right, what is the state of things. As far as I can tell

  • trunk is java17+ only
  • @stoty has been running locally and not seeing any more failures than trunk
  • lots of reviews including by @szetszwo @slfan1989, @cnauroth and more
  • it touches enough files that every week it's not merged there's rebase pain.

I think we should merge it unless there's a veto from some of the reviewers. I don't want to make code quality worse, and the JUnit 5 move was fairly hard work for all -but this is being forced on us by oracle

@stoty
Copy link
Contributor Author

stoty commented Nov 3, 2025

Thanks @steveloughran
Please use #7919 instead which is split into two commits (remove the third one), and which I have just rebased.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

10 participants