diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/SubjectUtil.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/SubjectUtil.java index faf2d6c7d8131..e364f04059620 100644 --- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/SubjectUtil.java +++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/SubjectUtil.java @@ -56,6 +56,18 @@ public final class SubjectUtil { HAS_CALL_AS ? null : lookupDoAsThrowException(); private static final MethodHandle CURRENT = lookupCurrent(); + // copied from org.apache.hadoop.util.Shell to break circular dependency + // "1.8"->8, "9"->9, "10"->10 + private static final int JAVA_SPEC_VER = Math.max(8, + Integer.parseInt(System.getProperty("java.specification.version").split("\\.")[0])); + + public static final boolean THREAD_INHERITS_SUBJECT = checkThreadInheritsSubject(); + + /** + * Try to return the method handle for Subject#callAs() + * + * @return the method handle, or null if the Java version does not have it + */ private static MethodHandle lookupCallAs() { MethodHandles.Lookup lookup = MethodHandles.lookup(); try { @@ -71,6 +83,38 @@ private static MethodHandle lookupCallAs() { } } + /** + * Determine whether we need to explicitly propagate the Subject into new Threads. + * + * @return true if new Threads inherit the Subject from the parent + */ + private static boolean checkThreadInheritsSubject() { + + boolean securityManagerEnabled = true; + try { + // TODO this needs SecurityManager to compile, use reflection to look it up instead + SecurityManager sm = System.getSecurityManager(); + System.setSecurityManager(sm); + } catch (UnsupportedOperationException e) { + // JDK24+ unconditionally throws this, so we don't need to check for JDK24+ + // explicitly + securityManagerEnabled = false; + } catch (Throwable t) { + // don't care + } + + return JAVA_SPEC_VER < 22 || securityManagerEnabled; + } + + /** + * Look up the method handle for Subject#doAs(PrivilegedAction) + * + * This is only called if Subject#callAs() does not exist. + * If we can't fall back to doAs(), that's a hard error. + * + * @return the method handle + * @throws ExceptionInInitializerError if unable to get the method handle + */ private static MethodHandle lookupDoAs() { MethodHandles.Lookup lookup = MethodHandles.lookup(); try { @@ -82,6 +126,15 @@ private static MethodHandle lookupDoAs() { } } + /** + * Look up the method handle for Subject#doAs(PrivilegedExceptionAction) + * + * This is only called if Subject#callAs() does not exist. + * If we can't fall back to doAs(), that's a hard error. + * + * @return the method handle + * @throws ExceptionInInitializerError if unable to get the method handle + */ private static MethodHandle lookupDoAsThrowException() { MethodHandles.Lookup lookup = MethodHandles.lookup(); try { @@ -93,6 +146,15 @@ private static MethodHandle lookupDoAsThrowException() { } } + /** + * Look up the method handle for Subject#current(). + * + * If Subject#current() is not present, fall back to returning + * a method handle for Subject.getSubject(AccessController.getContext()) + * + * @return the method handle or null if it does not exist + * @throws ExceptionInInitializerError if neither current() nor the fallback is found + */ private static MethodHandle lookupCurrent() { MethodHandles.Lookup lookup = MethodHandles.lookup(); try { @@ -112,6 +174,15 @@ private static MethodHandle lookupCurrent() { } } + /** + * Look up the method handle for Subject#getSubject(AccessControlContext) + * + * This is only called if Subject#current() does not exist. + * If we can't fall back to getSubject(), that's a hard error. + * + * @return the method handle + * @throws ExceptionInInitializerError if cannot get the handle + */ private static MethodHandle lookupGetSubject() { MethodHandles.Lookup lookup = MethodHandles.lookup(); try { @@ -124,6 +195,15 @@ private static MethodHandle lookupGetSubject() { } } + /** + * Look up the method handle for AccessController.getAccessControlContext() + * + * This is only called if Subject#current() does not exist. + * If we can't find this method, then we can't fall back which is hard error. + * + * @return the method handle + * @throws ExceptionInInitializerError if cannot get the handle + */ private static MethodHandle lookupGetContext() { try { // Use reflection to work with Java versions that have and don't have @@ -264,6 +344,13 @@ public static Subject current() { } } + /** + * Convert a Callable into a PrivilegedAction + * + * @param return type + * @param callable to be converted + * @return PrivilegedAction wrapping the callable + */ private static PrivilegedAction callableToPrivilegedAction( Callable callable) { return () -> { @@ -275,11 +362,25 @@ private static PrivilegedAction callableToPrivilegedAction( }; } + /** + * Convert a PrivilegedExceptionAction into a Callable + * + * @param return type + * @param action to be wrapped + * @return Callable wrapping the action + */ private static Callable privilegedExceptionActionToCallable( PrivilegedExceptionAction action) { return action::run; } + /** + * Convert a PrivilegedAction into a Callable + * + * @param return type + * @param action to be wrapped + * @return Callable wrapping the action + */ private static Callable privilegedActionToCallable( PrivilegedAction action) { return action::run; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HealthMonitor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HealthMonitor.java index d222d52e37349..d37c321ff4311 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HealthMonitor.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HealthMonitor.java @@ -283,7 +283,7 @@ public void uncaughtException(Thread t, Throwable e) { } @Override - public void run() { + public void work() { while (shouldRun) { try { loopUntilConnected(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/AsyncCallHandler.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/AsyncCallHandler.java index 60210ccd920c2..010f4928be9ff 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/AsyncCallHandler.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/AsyncCallHandler.java @@ -158,7 +158,7 @@ void tryStart() { if (running.compareAndSet(null, current)) { final Daemon daemon = new Daemon() { @Override - public void run() { + public void work() { for (; isRunning(this);) { final long waitTime = checkCalls(); tryStop(this); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Daemon.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Daemon.java index f735b82e4289b..88ac215e4c32b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Daemon.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Daemon.java @@ -18,25 +18,74 @@ package org.apache.hadoop.util; +import java.security.PrivilegedAction; import java.util.concurrent.ThreadFactory; +import javax.security.auth.Subject; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.security.authentication.util.SubjectUtil; -/** A thread that has called {@link Thread#setDaemon(boolean) } with true.*/ -@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"}) +/** + * A thread that has called {@link Thread#setDaemon(boolean) } with true. + *

+ * The runnable code must either be specified in the runnable parameter or in + * the overridden work() method. + *

+ * See {@link org.apache.hadoop.util.concurrent.SubjectInheritingThread} for the Subject inheritance behavior this + * class adds. + * + */ +@InterfaceAudience.LimitedPrivate({ "HDFS", "MapReduce" }) @InterfaceStability.Unstable public class Daemon extends Thread { + Subject startSubject; + + @Override + public final void start() { + if (!SubjectUtil.THREAD_INHERITS_SUBJECT) { + startSubject = SubjectUtil.current(); + } + super.start(); + } + + /** + * Override this instead of run() + */ + public void work() { + if (runnable != null) { + runnable.run(); + } + } + + @Override + public final void run() { + if (!SubjectUtil.THREAD_INHERITS_SUBJECT) { + SubjectUtil.doAs(startSubject, new PrivilegedAction() { + + @Override + public Void run() { + work(); + return null; + } + + }); + } else { + work(); + } + } + { - setDaemon(true); // always a daemon + setDaemon(true); // always a daemon } /** - * Provide a factory for named daemon threads, - * for use in ExecutorServices constructors + * Provide a factory for named daemon threads, for use in ExecutorServices + * constructors */ - @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"}) + @InterfaceAudience.LimitedPrivate({ "HDFS", "MapReduce" }) public static class DaemonFactory extends Daemon implements ThreadFactory { @Override @@ -47,6 +96,7 @@ public Thread newThread(Runnable runnable) { } Runnable runnable = null; + /** Construct a daemon thread. */ public Daemon() { super(); @@ -54,23 +104,25 @@ public Daemon() { /** * Construct a daemon thread. + * * @param runnable runnable. */ public Daemon(Runnable runnable) { super(runnable); this.runnable = runnable; - this.setName(((Object)runnable).toString()); + this.setName(((Object) runnable).toString()); } /** * Construct a daemon thread to be part of a specified thread group. - * @param group thread group. + * + * @param group thread group. * @param runnable runnable. */ public Daemon(ThreadGroup group, Runnable runnable) { super(group, runnable); this.runnable = runnable; - this.setName(((Object)runnable).toString()); + this.setName(((Object) runnable).toString()); } public Runnable getRunnable() { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/SubjectInheritingThread.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/SubjectInheritingThread.java new file mode 100644 index 0000000000000..e9b6745340d45 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/SubjectInheritingThread.java @@ -0,0 +1,212 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.util.concurrent; + +import java.security.PrivilegedAction; +import javax.security.auth.Subject; + +import org.apache.hadoop.security.authentication.util.SubjectUtil; + +/** + * Helper class to restore Subject propagation behavior of threads after the + * JEP411/JEP486 changes. + *

+ * Java propagates the current Subject to any new Threads in all version up to + * Java 21. In Java 22-23 the Subject is only propagated if the SecurityManager + * is enabled, while in Java 24+ it is never propagated. + *

+ * Hadoop security heavily relies on the original behavior, as Subject is at the + * core of JAAS. This class wraps thread. It overrides start() and saves the + * Subject of the current thread, and wraps the payload in a + * Subject.doAs()/callAs() call to restore it in the newly created Thread. + *

+ * When specifying a Runnable, this class is used in exactly the same way as + * Thread. + *

+ * {@link #run()} cannot be directly overridden, as that would also override the + * subject restoration logic. SubjectInheritingThread provides a {@link work()} + * method instead, which is wrapped and invoked by its own final {@link run()} + * method. + */ +public class SubjectInheritingThread extends Thread { + + private Subject startSubject; + // {@link Thread#target} is private, so we need our own + private Runnable hadoopTarget; + + /** + * Behaves similarly to {@link Thread#Thread()} constructor, but the code to run + * must be specified by overriding the {@link #work()} instead of the {link + * #run()} method. + */ + public SubjectInheritingThread() { + super(); + } + + /** + * Behaves similarly to {@link Thread#Thread(Runnable)} constructor. + * + * @param target the object whose {@code run} method is invoked when this thread + * is started. If {@code null}, this classes {@code run} method + * does nothing. + */ + public SubjectInheritingThread(Runnable target) { + super(); + this.hadoopTarget = target; + } + + /** + * Behaves similarly to {@link Thread#Thread(ThreadGroup, Runnable)} + * constructor. + * + * @param group the thread group. If {@code null} and there is a security + * manager, the group is determined by + * {@linkplain SecurityManager#getThreadGroup + * SecurityManager.getThreadGroup()}. If there is not a security + * manager or {@code + * SecurityManager.getThreadGroup()} returns {@code null}, the group is + * set to the current thread's thread group. + * + * @param target the object whose {@code run} method is invoked when this thread + * is started. If {@code null}, this thread's run method is + * invoked. + * @throws SecurityException if the current thread cannot create a thread in the + * specified thread group + */ + public SubjectInheritingThread(ThreadGroup group, Runnable target) { + // The target passed to Thread has no effect, we only pass it + // because there is no super(group) constructor. + super(group, target); + this.hadoopTarget = target; + } + + /** + * Behaves similarly to {@link Thread#Thread(Runnable, String)} constructor. + * + * @param target the object whose {@code run} method is invoked when this thread + * is started. If {@code null}, this thread's run method is + * invoked. + * + * @param name the name of the new thread + * + * @throws SecurityException if the current thread cannot create a thread in the + * specified thread group + */ + public SubjectInheritingThread(Runnable target, String name) { + super(name); + this.hadoopTarget = target; + } + + /** + * Behaves similarly to {@link Thread#Thread(String)} constructor. + * + * @param name the name of the new thread + */ + public SubjectInheritingThread(String name) { + super(name); + } + + /** + * Behaves similarly to {@link Thread#Thread(ThreadGroup, String)} constructor. + * + * @param group the thread group. If {@code null} and there is a security + * manager, the group is determined by + * {@linkplain SecurityManager#getThreadGroup + * SecurityManager.getThreadGroup()}. If there is not a security + * manager or {@code + * SecurityManager.getThreadGroup()} returns {@code null}, the group is + * set to the current thread's thread group. + * + * @param name the name of the new thread + */ + public SubjectInheritingThread(ThreadGroup group, String name) { + super(group, name); + } + + /** + * Behaves similarly to {@link Thread#Thread(ThreadGroup, Runnable, String)} + * constructor. + * + * @param group the thread group. If {@code null} and there is a security + * manager, the group is determined by + * {@linkplain SecurityManager#getThreadGroup + * SecurityManager.getThreadGroup()}. If there is not a security + * manager or {@code + * SecurityManager.getThreadGroup()} returns {@code null}, the group is + * set to the current thread's thread group. + * + * @param target the object whose {@code run} method is invoked when this thread + * is started. If {@code null}, this thread's run method is + * invoked. + * + * @param name the name of the new thread + * + * @throws SecurityException if the current thread cannot create a thread in the + * specified thread group or cannot override the + * context class loader methods. + */ + public SubjectInheritingThread(ThreadGroup group, Runnable target, String name) { + super(group, name); + this.hadoopTarget = target; + } + + /** + * Behaves similarly to pre-Java 22 {@link Thread#start()}. It saves the current + * Subject before starting the new thread, which is then used as the Subject for + * the Runnable or the overridden work() method. + */ + @Override + public final void start() { + if (!SubjectUtil.THREAD_INHERITS_SUBJECT) { + startSubject = SubjectUtil.current(); + } + super.start(); + } + + /** + * 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() { + if (hadoopTarget != null) { + hadoopTarget.run(); + } + } + + /** + * This cannot be overridden in this class. Override the {@link #work()} method + * instead which behaves like pre-Java 22 {@link Thread#run()} + */ + @Override + public final void run() { + if (!SubjectUtil.THREAD_INHERITS_SUBJECT) { + SubjectUtil.doAs(startSubject, new PrivilegedAction() { + + @Override + public Void run() { + work(); + return null; + } + + }); + } else { + work(); + } + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/concurrent/TestSubjectPropagation.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/concurrent/TestSubjectPropagation.java new file mode 100644 index 0000000000000..018a91cdfc239 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/concurrent/TestSubjectPropagation.java @@ -0,0 +1,186 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.util.concurrent; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; + +import java.util.concurrent.Callable; + +import javax.security.auth.Subject; + +import org.apache.hadoop.security.authentication.util.SubjectUtil; +import org.apache.hadoop.util.Daemon; +import org.apache.hadoop.util.Shell; +import org.junit.jupiter.api.Test; + +public class TestSubjectPropagation { + + private Subject childSubject = null; + + @Test + public void testSubjectInheritingThreadOverride() { + Subject parentSubject = new Subject(); + childSubject = null; + + SubjectUtil.callAs(parentSubject, new Callable() { + public Void call() throws InterruptedException { + SubjectInheritingThread t = new SubjectInheritingThread() { + @Override + public void work() { + childSubject = SubjectUtil.current(); + } + }; + t.start(); + t.join(1000); + return (Void) null; + } + }); + + assertEquals(parentSubject, childSubject); + } + + @Test + public void testSubjectInheritingThreadRunnable() { + Subject parentSubject = new Subject(); + childSubject = null; + + SubjectUtil.callAs(parentSubject, new Callable() { + public Void call() throws InterruptedException { + Runnable r = new Runnable() { + @Override + public void run() { + childSubject = SubjectUtil.current(); + } + }; + + SubjectInheritingThread t = new SubjectInheritingThread(r); + t.start(); + t.join(1000); + return (Void) null; + } + }); + + assertEquals(parentSubject, childSubject); + } + + @Test + public void testDaemonOverride() { + Subject parentSubject = new Subject(); + childSubject = null; + + SubjectUtil.callAs(parentSubject, new Callable() { + public Void call() throws InterruptedException { + Daemon t = new Daemon() { + @Override + public void work() { + childSubject = SubjectUtil.current(); + } + }; + t.start(); + t.join(1000); + return (Void) null; + } + }); + + assertEquals(parentSubject, childSubject); + } + + @Test + public void testDaemonRunnable() { + Subject parentSubject = new Subject(); + childSubject = null; + + SubjectUtil.callAs(parentSubject, new Callable() { + public Void call() throws InterruptedException { + Runnable r = new Runnable() { + @Override + public void run() { + childSubject = SubjectUtil.current(); + } + }; + + Daemon t = new Daemon(r); + t.start(); + t.join(1000); + return (Void) null; + } + }); + + assertEquals(parentSubject, childSubject); + } + + @Test + public void testThreadOverride() { + Subject parentSubject = new Subject(); + childSubject = null; + + SubjectUtil.callAs(parentSubject, new Callable() { + public Void call() throws InterruptedException { + + Thread t = new Thread() { + @Override + public void run() { + childSubject = SubjectUtil.current(); + } + }; + t.start(); + t.join(1000); + return (Void) null; + } + }); + + if (SubjectUtil.THREAD_INHERITS_SUBJECT) { + assertEquals(parentSubject, childSubject); + } else { + // This is the behaviour that breaks Hadoop authorization + assertNull(childSubject); + } + } + + @Test + public void testThreadRunnable() { + Subject parentSubject = new Subject(); + childSubject = null; + + SubjectUtil.callAs(parentSubject, new Callable() { + public Void call() throws InterruptedException { + Runnable r = new Runnable() { + @Override + public void run() { + childSubject = SubjectUtil.current(); + } + }; + + Thread t = new Thread(r); + t.start(); + t.join(1000); + return (Void) null; + } + }); + + if (SubjectUtil.THREAD_INHERITS_SUBJECT) { + assertEquals(parentSubject, childSubject); + } else { + // This is the behaviour that breaks Hadoop authorization + assertNull(childSubject); + } + } + +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java index 8d13640eadb18..7caa88d6d65f9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java @@ -693,7 +693,7 @@ private boolean shouldStop() { * and closes them. Any error recovery is also done by this thread. */ @Override - public void run() { + public void work() { TraceScope scope = null; while (!streamerClosed && dfsClient.clientRunning) { // if the Responder encountered an error, shutdown Responder @@ -1167,7 +1167,7 @@ private class ResponseProcessor extends Daemon { } @Override - public void run() { + public void work() { setName("ResponseProcessor for block " + block); PipelineAck ack = new PipelineAck(); diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DeadNodeDetector.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DeadNodeDetector.java index 465497ffb9b60..0dc1b1686b98d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DeadNodeDetector.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DeadNodeDetector.java @@ -250,7 +250,7 @@ public DeadNodeDetector(String name, Configuration conf) { } @Override - public void run() { + public void work() { while (!Thread.currentThread().isInterrupted()) { clearAndGetDetectedDeadNodes(); LOG.debug("Current detector state {}, the detected nodes: {}.", state, diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/LocatedBlocksRefresher.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/LocatedBlocksRefresher.java index 454d1f9cd93e4..be8fd94247abb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/LocatedBlocksRefresher.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/LocatedBlocksRefresher.java @@ -99,7 +99,7 @@ public Thread newThread(Runnable r) { } @Override - public void run() { + public void work() { while (!Thread.currentThread().isInterrupted()) { if (!waitForInterval()) { diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtxCache.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtxCache.java index 70ae4b29e9f96..591776e2090be 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtxCache.java +++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtxCache.java @@ -246,7 +246,7 @@ void shouldRun(boolean shouldRun) { } @Override - public void run() { + public void work() { while (shouldRun) { scan(streamTimeout); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java index d1f02c47e90b8..ed29578ee0cf9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java @@ -3960,7 +3960,7 @@ public void processMisReplicatedBlocks() { reconstructionQueuesInitializer = new Daemon() { @Override - public void run() { + public void work() { try { processMisReplicatesAsync(); } catch (InterruptedException ie) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java index 29b262598bf55..fb63a13fb33b8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java @@ -128,7 +128,7 @@ void shutdown() { // The main work loop // @Override - public void run() { + public void work() { // How often to check the size of the edit log (min of checkpointCheckPeriod and checkpointPeriod) long periodMSec = checkpointConf.getCheckPeriod() * 1000; // How often to checkpoint regardless of number of txns diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestSpaceReservation.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestSpaceReservation.java index ef84c1732d5a9..bbb3c0552354a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestSpaceReservation.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestSpaceReservation.java @@ -552,7 +552,7 @@ private static class Writer extends Daemon { } @Override - public void run() { + public void work() { /** * Create a file, write up to 3 blocks of data and close the file. * Do this in a loop until we are told to stop.