|
@@ -40,7 +40,6 @@ import java.util.Set;
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
import java.util.concurrent.Executor;
|
|
|
import java.util.concurrent.locks.Condition;
|
|
|
-import java.util.concurrent.locks.ReentrantLock;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
import javax.management.NotCompliantMBeanException;
|
|
@@ -112,7 +111,7 @@ import org.apache.hadoop.util.Daemon;
|
|
|
import org.apache.hadoop.util.DataChecksum;
|
|
|
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
|
|
|
import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
|
|
|
-import org.apache.hadoop.util.InstrumentedLock;
|
|
|
+import org.apache.hadoop.util.InstrumentedReadWriteLock;
|
|
|
import org.apache.hadoop.util.ReflectionUtils;
|
|
|
import org.apache.hadoop.util.Time;
|
|
|
import org.apache.hadoop.util.Timer;
|
|
@@ -179,7 +178,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
|
|
|
@Override
|
|
|
public FsVolumeImpl getVolume(final ExtendedBlock b) {
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetReadLock.acquire()) {
|
|
|
final ReplicaInfo r =
|
|
|
volumeMap.get(b.getBlockPoolId(), b.getLocalBlock());
|
|
|
return r != null ? (FsVolumeImpl) r.getVolume() : null;
|
|
@@ -189,7 +188,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
@Override // FsDatasetSpi
|
|
|
public Block getStoredBlock(String bpid, long blkid)
|
|
|
throws IOException {
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetReadLock.acquire()) {
|
|
|
ReplicaInfo r = volumeMap.get(bpid, blkid);
|
|
|
if (r == null) {
|
|
|
return null;
|
|
@@ -202,12 +201,16 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
* The deepCopyReplica call doesn't use the datasetock since it will lead the
|
|
|
* potential deadlock with the {@link FsVolumeList#addBlockPool} call.
|
|
|
*/
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
@Override
|
|
|
public Set<? extends Replica> deepCopyReplica(String bpid)
|
|
|
throws IOException {
|
|
|
- Set<? extends Replica> replicas =
|
|
|
- new HashSet<>(volumeMap.replicas(bpid) == null ? Collections.EMPTY_SET
|
|
|
- : volumeMap.replicas(bpid));
|
|
|
+ Set<? extends Replica> replicas;
|
|
|
+ try (AutoCloseableLock lock = datasetReadLock.acquire()) {
|
|
|
+ replicas =
|
|
|
+ new HashSet<>(volumeMap.replicas(bpid) == null ? Collections.EMPTY_SET
|
|
|
+ : volumeMap.replicas(bpid));
|
|
|
+ }
|
|
|
return Collections.unmodifiableSet(replicas);
|
|
|
}
|
|
|
|
|
@@ -268,8 +271,12 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
private final int maxDataLength;
|
|
|
|
|
|
@VisibleForTesting
|
|
|
- final AutoCloseableLock datasetLock;
|
|
|
- private final Condition datasetLockCondition;
|
|
|
+ final AutoCloseableLock datasetWriteLock;
|
|
|
+ @VisibleForTesting
|
|
|
+ final AutoCloseableLock datasetReadLock;
|
|
|
+ @VisibleForTesting
|
|
|
+ final InstrumentedReadWriteLock datasetRWLock;
|
|
|
+ private final Condition datasetWriteLockCondition;
|
|
|
private static String blockPoolId = "";
|
|
|
|
|
|
/**
|
|
@@ -282,15 +289,33 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
this.dataStorage = storage;
|
|
|
this.conf = conf;
|
|
|
this.smallBufferSize = DFSUtilClient.getSmallBufferSize(conf);
|
|
|
- this.datasetLock = new AutoCloseableLock(
|
|
|
- new InstrumentedLock(getClass().getName(), LOG,
|
|
|
- new ReentrantLock(true),
|
|
|
- conf.getTimeDuration(
|
|
|
- DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_KEY,
|
|
|
- DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_DEFAULT,
|
|
|
- TimeUnit.MILLISECONDS),
|
|
|
- 300));
|
|
|
- this.datasetLockCondition = datasetLock.newCondition();
|
|
|
+ this.datasetRWLock = new InstrumentedReadWriteLock(
|
|
|
+ conf.getBoolean(DFSConfigKeys.DFS_DATANODE_LOCK_FAIR_KEY,
|
|
|
+ DFSConfigKeys.DFS_DATANODE_LOCK_FAIR_DEFAULT),
|
|
|
+ "FsDatasetRWLock", LOG, conf.getTimeDuration(
|
|
|
+ DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_KEY,
|
|
|
+ DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_DEFAULT,
|
|
|
+ TimeUnit.MILLISECONDS),
|
|
|
+ conf.getTimeDuration(
|
|
|
+ DFSConfigKeys.DFS_DATANODE_LOCK_REPORTING_THRESHOLD_MS_KEY,
|
|
|
+ DFSConfigKeys.DFS_DATANODE_LOCK_REPORTING_THRESHOLD_MS_DEFAULT,
|
|
|
+ TimeUnit.MILLISECONDS));
|
|
|
+ this.datasetWriteLock = new AutoCloseableLock(datasetRWLock.writeLock());
|
|
|
+ boolean enableRL = conf.getBoolean(
|
|
|
+ DFSConfigKeys.DFS_DATANODE_LOCK_READ_WRITE_ENABLED_KEY,
|
|
|
+ DFSConfigKeys.DFS_DATANODE_LOCK_READ_WRITE_ENABLED_DEFAULT);
|
|
|
+ // The read lock can be disabled by the above config key. If it is disabled
|
|
|
+ // then we simply make the both the read and write lock variables hold
|
|
|
+ // the write lock. All accesses to the lock are via these variables, so that
|
|
|
+ // effectively disables the read lock.
|
|
|
+ if (enableRL) {
|
|
|
+ LOG.info("The datanode lock is a read write lock");
|
|
|
+ this.datasetReadLock = new AutoCloseableLock(datasetRWLock.readLock());
|
|
|
+ } else {
|
|
|
+ LOG.info("The datanode lock is an exclusive write lock");
|
|
|
+ this.datasetReadLock = this.datasetWriteLock;
|
|
|
+ }
|
|
|
+ this.datasetWriteLockCondition = datasetWriteLock.newCondition();
|
|
|
|
|
|
// The number of volumes required for operation is the total number
|
|
|
// of volumes minus the number of failed volumes we can tolerate.
|
|
@@ -329,7 +354,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
}
|
|
|
|
|
|
storageMap = new ConcurrentHashMap<String, DatanodeStorage>();
|
|
|
- volumeMap = new ReplicaMap(datasetLock);
|
|
|
+ volumeMap = new ReplicaMap(datasetReadLock, datasetWriteLock);
|
|
|
ramDiskReplicaTracker = RamDiskReplicaTracker.getInstance(conf, this);
|
|
|
|
|
|
@SuppressWarnings("unchecked")
|
|
@@ -383,7 +408,12 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
|
|
|
@Override
|
|
|
public AutoCloseableLock acquireDatasetLock() {
|
|
|
- return datasetLock.acquire();
|
|
|
+ return datasetWriteLock.acquire();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public AutoCloseableLock acquireDatasetReadLock() {
|
|
|
+ return datasetReadLock.acquire();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -424,7 +454,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
ReplicaMap replicaMap,
|
|
|
Storage.StorageDirectory sd, StorageType storageType,
|
|
|
FsVolumeReference ref) throws IOException {
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
DatanodeStorage dnStorage = storageMap.get(sd.getStorageUuid());
|
|
|
if (dnStorage != null) {
|
|
|
final String errorMsg = String.format(
|
|
@@ -457,7 +487,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
.setConf(this.conf)
|
|
|
.build();
|
|
|
FsVolumeReference ref = fsVolume.obtainReference();
|
|
|
- ReplicaMap tempVolumeMap = new ReplicaMap(datasetLock);
|
|
|
+ ReplicaMap tempVolumeMap =
|
|
|
+ new ReplicaMap(datasetReadLock, datasetWriteLock);
|
|
|
fsVolume.getVolumeMap(tempVolumeMap, ramDiskReplicaTracker);
|
|
|
|
|
|
activateVolume(tempVolumeMap, sd, storageLocation.getStorageType(), ref);
|
|
@@ -496,7 +527,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
StorageType storageType = location.getStorageType();
|
|
|
final FsVolumeImpl fsVolume =
|
|
|
createFsVolume(sd.getStorageUuid(), sd, location);
|
|
|
- final ReplicaMap tempVolumeMap = new ReplicaMap(new AutoCloseableLock());
|
|
|
+ final ReplicaMap tempVolumeMap =
|
|
|
+ new ReplicaMap(datasetReadLock, datasetWriteLock);
|
|
|
ArrayList<IOException> exceptions = Lists.newArrayList();
|
|
|
|
|
|
for (final NamespaceInfo nsInfo : nsInfos) {
|
|
@@ -541,7 +573,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
new ArrayList<>(storageLocsToRemove);
|
|
|
Map<String, List<ReplicaInfo>> blkToInvalidate = new HashMap<>();
|
|
|
List<String> storageToRemove = new ArrayList<>();
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
for (int idx = 0; idx < dataStorage.getNumStorageDirs(); idx++) {
|
|
|
Storage.StorageDirectory sd = dataStorage.getStorageDir(idx);
|
|
|
final StorageLocation sdLocation = sd.getStorageLocation();
|
|
@@ -553,7 +585,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
// Disable the volume from the service.
|
|
|
asyncDiskService.removeVolume(sd.getStorageUuid());
|
|
|
volumes.removeVolume(sdLocation, clearFailure);
|
|
|
- volumes.waitVolumeRemoved(5000, datasetLockCondition);
|
|
|
+ volumes.waitVolumeRemoved(5000, datasetWriteLockCondition);
|
|
|
|
|
|
// Removed all replica information for the blocks on the volume.
|
|
|
// Unlike updating the volumeMap in addVolume(), this operation does
|
|
@@ -600,7 +632,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
for(String storageUuid : storageToRemove) {
|
|
|
storageMap.remove(storageUuid);
|
|
|
}
|
|
@@ -791,7 +823,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
long seekOffset) throws IOException {
|
|
|
|
|
|
ReplicaInfo info;
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetReadLock.acquire()) {
|
|
|
info = volumeMap.get(b.getBlockPoolId(), b.getLocalBlock());
|
|
|
}
|
|
|
|
|
@@ -879,7 +911,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
@Override // FsDatasetSpi
|
|
|
public ReplicaInputStreams getTmpInputStreams(ExtendedBlock b,
|
|
|
long blkOffset, long metaOffset) throws IOException {
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetReadLock.acquire()) {
|
|
|
ReplicaInfo info = getReplicaInfo(b);
|
|
|
FsVolumeReference ref = info.getVolume().obtainReference();
|
|
|
try {
|
|
@@ -1004,7 +1036,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
}
|
|
|
|
|
|
FsVolumeReference volumeRef = null;
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetReadLock.acquire()) {
|
|
|
volumeRef = volumes.getNextVolume(targetStorageType, targetStorageId,
|
|
|
block.getNumBytes());
|
|
|
}
|
|
@@ -1118,7 +1150,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
|
|
|
FsVolumeReference volumeRef = null;
|
|
|
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetReadLock.acquire()) {
|
|
|
volumeRef = destination.obtainReference();
|
|
|
}
|
|
|
|
|
@@ -1206,7 +1238,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
@Override // FsDatasetSpi
|
|
|
public ReplicaHandler append(ExtendedBlock b,
|
|
|
long newGS, long expectedBlockLen) throws IOException {
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
// If the block was successfully finalized because all packets
|
|
|
// were successfully processed at the Datanode but the ack for
|
|
|
// some of the packets were not received by the client. The client
|
|
@@ -1258,7 +1290,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
private ReplicaInPipeline append(String bpid,
|
|
|
ReplicaInfo replicaInfo, long newGS, long estimateBlockLen)
|
|
|
throws IOException {
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
// If the block is cached, start uncaching it.
|
|
|
if (replicaInfo.getState() != ReplicaState.FINALIZED) {
|
|
|
throw new IOException("Only a Finalized replica can be appended to; "
|
|
@@ -1354,7 +1386,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
|
|
|
while (true) {
|
|
|
try {
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
ReplicaInfo replicaInfo = recoverCheck(b, newGS, expectedBlockLen);
|
|
|
FsVolumeReference ref = replicaInfo.getVolume().obtainReference();
|
|
|
ReplicaInPipeline replica;
|
|
@@ -1386,7 +1418,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
LOG.info("Recover failed close " + b);
|
|
|
while (true) {
|
|
|
try {
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
// check replica's state
|
|
|
ReplicaInfo replicaInfo = recoverCheck(b, newGS, expectedBlockLen);
|
|
|
// bump the replica's GS
|
|
@@ -1408,7 +1440,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
public ReplicaHandler createRbw(
|
|
|
StorageType storageType, String storageId, ExtendedBlock b,
|
|
|
boolean allowLazyPersist) throws IOException {
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
ReplicaInfo replicaInfo = volumeMap.get(b.getBlockPoolId(),
|
|
|
b.getBlockId());
|
|
|
if (replicaInfo != null) {
|
|
@@ -1479,7 +1511,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
|
|
|
while (true) {
|
|
|
try {
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
ReplicaInfo replicaInfo =
|
|
|
getReplicaInfo(b.getBlockPoolId(), b.getBlockId());
|
|
|
// check the replica's state
|
|
@@ -1504,7 +1536,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
private ReplicaHandler recoverRbwImpl(ReplicaInPipeline rbw,
|
|
|
ExtendedBlock b, long newGS, long minBytesRcvd, long maxBytesRcvd)
|
|
|
throws IOException {
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
// check generation stamp
|
|
|
long replicaGenerationStamp = rbw.getGenerationStamp();
|
|
|
if (replicaGenerationStamp < b.getGenerationStamp() ||
|
|
@@ -1565,7 +1597,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
public ReplicaInPipeline convertTemporaryToRbw(
|
|
|
final ExtendedBlock b) throws IOException {
|
|
|
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
final long blockId = b.getBlockId();
|
|
|
final long expectedGs = b.getGenerationStamp();
|
|
|
final long visible = b.getNumBytes();
|
|
@@ -1639,7 +1671,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
ReplicaInfo lastFoundReplicaInfo = null;
|
|
|
boolean isInPipeline = false;
|
|
|
do {
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
ReplicaInfo currentReplicaInfo =
|
|
|
volumeMap.get(b.getBlockPoolId(), b.getBlockId());
|
|
|
if (currentReplicaInfo == lastFoundReplicaInfo) {
|
|
@@ -1692,7 +1724,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
invalidate(b.getBlockPoolId(), new Block[] { lastFoundReplicaInfo },
|
|
|
false);
|
|
|
}
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
FsVolumeReference ref = volumes.getNextVolume(storageType, storageId, b
|
|
|
.getNumBytes());
|
|
|
FsVolumeImpl v = (FsVolumeImpl) ref.getVolume();
|
|
@@ -1743,7 +1775,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
throws IOException {
|
|
|
ReplicaInfo replicaInfo = null;
|
|
|
ReplicaInfo finalizedReplicaInfo = null;
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
if (Thread.interrupted()) {
|
|
|
// Don't allow data modifications from interrupted threads
|
|
|
throw new IOException("Cannot finalize block from Interrupted Thread");
|
|
@@ -1774,7 +1806,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
|
|
|
private ReplicaInfo finalizeReplica(String bpid, ReplicaInfo replicaInfo)
|
|
|
throws IOException {
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
// Compare generation stamp of old and new replica before finalizing
|
|
|
if (volumeMap.get(bpid, replicaInfo.getBlockId()).getGenerationStamp()
|
|
|
> replicaInfo.getGenerationStamp()) {
|
|
@@ -1819,7 +1851,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
*/
|
|
|
@Override // FsDatasetSpi
|
|
|
public void unfinalizeBlock(ExtendedBlock b) throws IOException {
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
ReplicaInfo replicaInfo = volumeMap.get(b.getBlockPoolId(),
|
|
|
b.getLocalBlock());
|
|
|
if (replicaInfo != null &&
|
|
@@ -1872,7 +1904,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
new HashMap<String, BlockListAsLongs.Builder>();
|
|
|
|
|
|
List<FsVolumeImpl> curVolumes = null;
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetReadLock.acquire()) {
|
|
|
curVolumes = volumes.getVolumes();
|
|
|
for (FsVolumeSpi v : curVolumes) {
|
|
|
builders.put(v.getStorageID(), BlockListAsLongs.builder(maxDataLength));
|
|
@@ -1927,7 +1959,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
* Gets a list of references to the finalized blocks for the given block pool.
|
|
|
* <p>
|
|
|
* Callers of this function should call
|
|
|
- * {@link FsDatasetSpi#acquireDatasetLock} to avoid blocks' status being
|
|
|
+ * {@link FsDatasetSpi#acquireDatasetLock()} to avoid blocks' status being
|
|
|
* changed during list iteration.
|
|
|
* </p>
|
|
|
* @return a list of references to the finalized blocks for the given block
|
|
@@ -1935,7 +1967,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
*/
|
|
|
@Override
|
|
|
public List<ReplicaInfo> getFinalizedBlocks(String bpid) {
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetReadLock.acquire()) {
|
|
|
final List<ReplicaInfo> finalized = new ArrayList<ReplicaInfo>(
|
|
|
volumeMap.size(bpid));
|
|
|
for (ReplicaInfo b : volumeMap.replicas(bpid)) {
|
|
@@ -2028,9 +2060,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
ReplicaInfo validateBlockFile(String bpid, long blockId) {
|
|
|
//Should we check for metadata file too?
|
|
|
final ReplicaInfo r;
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
- r = volumeMap.get(bpid, blockId);
|
|
|
- }
|
|
|
+ r = volumeMap.get(bpid, blockId);
|
|
|
if (r != null) {
|
|
|
if (r.blockDataExists()) {
|
|
|
return r;
|
|
@@ -2079,7 +2109,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
for (int i = 0; i < invalidBlks.length; i++) {
|
|
|
final ReplicaInfo removing;
|
|
|
final FsVolumeImpl v;
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
final ReplicaInfo info = volumeMap.get(bpid, invalidBlks[i]);
|
|
|
if (info == null) {
|
|
|
ReplicaInfo infoByBlockId =
|
|
@@ -2205,7 +2235,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
long length, genstamp;
|
|
|
Executor volumeExecutor;
|
|
|
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
ReplicaInfo info = volumeMap.get(bpid, blockId);
|
|
|
boolean success = false;
|
|
|
try {
|
|
@@ -2273,7 +2303,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
|
|
|
@Override // FsDatasetSpi
|
|
|
public boolean contains(final ExtendedBlock block) {
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetReadLock.acquire()) {
|
|
|
final long blockId = block.getLocalBlock().getBlockId();
|
|
|
final String bpid = block.getBlockPoolId();
|
|
|
final ReplicaInfo r = volumeMap.get(bpid, blockId);
|
|
@@ -2393,7 +2423,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
|
|
|
Block corruptBlock = null;
|
|
|
ReplicaInfo memBlockInfo;
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
memBlockInfo = volumeMap.get(bpid, blockId);
|
|
|
if (memBlockInfo != null &&
|
|
|
memBlockInfo.getState() != ReplicaState.FINALIZED) {
|
|
@@ -2594,7 +2624,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
|
|
|
@Override
|
|
|
public String getReplicaString(String bpid, long blockId) {
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetReadLock.acquire()) {
|
|
|
final Replica r = volumeMap.get(bpid, blockId);
|
|
|
return r == null ? "null" : r.toString();
|
|
|
}
|
|
@@ -2701,7 +2731,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
final long recoveryId,
|
|
|
final long newBlockId,
|
|
|
final long newlength) throws IOException {
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
//get replica
|
|
|
final String bpid = oldBlock.getBlockPoolId();
|
|
|
final ReplicaInfo replica = volumeMap.get(bpid, oldBlock.getBlockId());
|
|
@@ -2814,7 +2844,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
@Override // FsDatasetSpi
|
|
|
public long getReplicaVisibleLength(final ExtendedBlock block)
|
|
|
throws IOException {
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetReadLock.acquire()) {
|
|
|
final Replica replica = getReplicaInfo(block.getBlockPoolId(),
|
|
|
block.getBlockId());
|
|
|
if (replica.getGenerationStamp() < block.getGenerationStamp()) {
|
|
@@ -2831,7 +2861,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
throws IOException {
|
|
|
LOG.info("Adding block pool " + bpid);
|
|
|
AddBlockPoolException volumeExceptions = new AddBlockPoolException();
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
try {
|
|
|
volumes.addBlockPool(bpid, conf);
|
|
|
} catch (AddBlockPoolException e) {
|
|
@@ -2861,7 +2891,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
|
|
|
@Override
|
|
|
public void shutdownBlockPool(String bpid) {
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
LOG.info("Removing block pool " + bpid);
|
|
|
Map<DatanodeStorage, BlockListAsLongs> blocksPerVolume
|
|
|
= getBlockReports(bpid);
|
|
@@ -2935,7 +2965,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
@Override //FsDatasetSpi
|
|
|
public void deleteBlockPool(String bpid, boolean force)
|
|
|
throws IOException {
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
List<FsVolumeImpl> curVolumes = volumes.getVolumes();
|
|
|
if (!force) {
|
|
|
for (FsVolumeImpl volume : curVolumes) {
|
|
@@ -2964,18 +2994,20 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
@Override // FsDatasetSpi
|
|
|
public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block)
|
|
|
throws IOException {
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetReadLock.acquire()) {
|
|
|
final Replica replica = volumeMap.get(block.getBlockPoolId(),
|
|
|
block.getBlockId());
|
|
|
if (replica == null) {
|
|
|
throw new ReplicaNotFoundException(block);
|
|
|
}
|
|
|
- if (replica.getGenerationStamp() < block.getGenerationStamp()) {
|
|
|
- throw new IOException(
|
|
|
- "Replica generation stamp < block generation stamp, block="
|
|
|
- + block + ", replica=" + replica);
|
|
|
- } else if (replica.getGenerationStamp() > block.getGenerationStamp()) {
|
|
|
- block.setGenerationStamp(replica.getGenerationStamp());
|
|
|
+ synchronized(replica) {
|
|
|
+ if (replica.getGenerationStamp() < block.getGenerationStamp()) {
|
|
|
+ throw new IOException(
|
|
|
+ "Replica generation stamp < block generation stamp, block="
|
|
|
+ + block + ", replica=" + replica);
|
|
|
+ } else if (replica.getGenerationStamp() > block.getGenerationStamp()) {
|
|
|
+ block.setGenerationStamp(replica.getGenerationStamp());
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -3016,7 +3048,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
@Override
|
|
|
public void onCompleteLazyPersist(String bpId, long blockId,
|
|
|
long creationTime, File[] savedFiles, FsVolumeImpl targetVolume) {
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
ramDiskReplicaTracker.recordEndLazyPersist(bpId, blockId, savedFiles);
|
|
|
|
|
|
targetVolume.incDfsUsedAndNumBlocks(bpId, savedFiles[0].length()
|
|
@@ -3150,7 +3182,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
try {
|
|
|
block = ramDiskReplicaTracker.dequeueNextReplicaToPersist();
|
|
|
if (block != null) {
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
replicaInfo = volumeMap.get(block.getBlockPoolId(), block.getBlockId());
|
|
|
|
|
|
// If replicaInfo is null, the block was either deleted before
|
|
@@ -3217,7 +3249,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
ReplicaInfo replicaInfo, newReplicaInfo;
|
|
|
final String bpid = replicaState.getBlockPoolId();
|
|
|
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
replicaInfo = getReplicaInfo(replicaState.getBlockPoolId(),
|
|
|
replicaState.getBlockId());
|
|
|
Preconditions.checkState(replicaInfo.getVolume().isTransientStorage());
|
|
@@ -3390,7 +3422,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
}
|
|
|
|
|
|
void stopAllDataxceiverThreads(FsVolumeImpl volume) {
|
|
|
- try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
|
+ try (AutoCloseableLock lock = datasetWriteLock.acquire()) {
|
|
|
for (String bpid : volumeMap.getBlockPoolList()) {
|
|
|
Collection<ReplicaInfo> replicas = volumeMap.replicas(bpid);
|
|
|
for (ReplicaInfo replicaInfo : replicas) {
|