Skip to content

Commit

Permalink
HDFS-17691. [FGL] Move FSNamesystemLockMode to org.apache.hadoop.hdfs…
Browse files Browse the repository at this point in the history
….util package (#7232)
  • Loading branch information
ZanderXu authored Dec 24, 2024
1 parent 1e7c88f commit 60f3fca
Show file tree
Hide file tree
Showing 87 changed files with 937 additions and 938 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -121,13 +121,13 @@
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
import org.apache.hadoop.hdfs.server.namenode.fgl.FSNamesystemLockMode;
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.util.RwLockMode;
import org.apache.hadoop.io.EnumSetWritable;
import org.apache.hadoop.io.erasurecode.ECSchema;
import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
Expand Down Expand Up @@ -1699,10 +1699,10 @@ public void testGetReplicatedBlockStats() throws Exception {
// mark a replica as corrupt
LocatedBlock block = NameNodeAdapter
.getBlockLocations(nameNode, testFile, 0, 1024).get(0);
namesystem.writeLock(FSNamesystemLockMode.BM);
namesystem.writeLock(RwLockMode.BM);
bm.findAndMarkBlockAsCorrupt(block.getBlock(), block.getLocations()[0],
"STORAGE_ID", "TEST");
namesystem.writeUnlock(FSNamesystemLockMode.BM, "findAndMarkBlockAsCorrupt");
namesystem.writeUnlock(RwLockMode.BM, "findAndMarkBlockAsCorrupt");
BlockManagerTestUtil.updateState(bm);
DFSTestUtil.waitCorruptReplicas(fileSystem, namesystem,
new Path(testFile), block.getBlock(), 1);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@
import java.util.List;
import java.util.Map.Entry;

import org.apache.hadoop.hdfs.server.namenode.fgl.FSNamesystemLockMode;
import org.apache.hadoop.util.Lists;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand All @@ -41,6 +40,7 @@
import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Counter;
import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
import org.apache.hadoop.hdfs.server.namenode.startupprogress.StepType;
import org.apache.hadoop.hdfs.util.RwLockMode;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.ipc.RetriableException;
import org.apache.hadoop.ipc.StandbyException;
Expand Down Expand Up @@ -373,7 +373,7 @@ protected void logUpdateMasterKey(DelegationKey key)
// closes the edit log files. Doing this inside the
// fsn lock will prevent being interrupted when stopping
// the secret manager.
namesystem.readLockInterruptibly(FSNamesystemLockMode.FS);
namesystem.readLockInterruptibly(RwLockMode.FS);
try {
// this monitor isn't necessary if stopped while holding write lock
// but for safety, guard against a stop with read lock.
Expand All @@ -384,7 +384,7 @@ protected void logUpdateMasterKey(DelegationKey key)
namesystem.logUpdateMasterKey(key);
}
} finally {
namesystem.readUnlock(FSNamesystemLockMode.FS, "logUpdateMasterKey");
namesystem.readUnlock(RwLockMode.FS, "logUpdateMasterKey");
}
} catch (InterruptedException ie) {
// AbstractDelegationTokenManager may crash if an exception is thrown.
Expand All @@ -402,7 +402,7 @@ protected void logExpireToken(final DelegationTokenIdentifier dtId)
// closes the edit log files. Doing this inside the
// fsn lock will prevent being interrupted when stopping
// the secret manager.
namesystem.readLockInterruptibly(FSNamesystemLockMode.FS);
namesystem.readLockInterruptibly(RwLockMode.FS);
try {
// this monitor isn't necessary if stopped while holding write lock
// but for safety, guard against a stop with read lock.
Expand All @@ -413,7 +413,7 @@ protected void logExpireToken(final DelegationTokenIdentifier dtId)
namesystem.logExpireDelegationToken(dtId);
}
} finally {
namesystem.readUnlock(FSNamesystemLockMode.FS, "logExpireToken");
namesystem.readUnlock(RwLockMode.FS, "logExpireToken");
}
} catch (InterruptedException ie) {
// AbstractDelegationTokenManager may crash if an exception is thrown.
Expand Down
Loading

0 comments on commit 60f3fca

Please sign in to comment.