-
Notifications
You must be signed in to change notification settings - Fork 9.2k
BPServiceActor to provide new thread to handle IBR #4820
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -33,9 +33,13 @@ | |
| import java.util.SortedSet; | ||
| import java.util.TreeSet; | ||
| import java.util.concurrent.CountDownLatch; | ||
| import java.util.concurrent.ExecutorService; | ||
| import java.util.concurrent.Executors; | ||
| import java.util.concurrent.ThreadLocalRandom; | ||
| import java.util.concurrent.atomic.AtomicBoolean; | ||
|
|
||
| import com.google.common.collect.Maps; | ||
| import com.google.common.util.concurrent.ThreadFactoryBuilder; | ||
| import org.apache.hadoop.classification.InterfaceAudience; | ||
| import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState; | ||
| import org.apache.hadoop.hdfs.client.BlockReportOptions; | ||
|
|
@@ -92,6 +96,8 @@ class BPServiceActor implements Runnable { | |
|
|
||
| volatile long lastCacheReport = 0; | ||
| private final Scheduler scheduler; | ||
| private final Object sendIBRLock; | ||
| private final ExecutorService ibrExecutorService; | ||
|
|
||
| Thread bpThread; | ||
| DatanodeProtocolClientSideTranslatorPB bpNamenode; | ||
|
|
@@ -101,6 +107,7 @@ static enum RunningState { | |
| } | ||
|
|
||
| private volatile RunningState runningState = RunningState.CONNECTING; | ||
| private volatile boolean sendImmediateIBR = false; | ||
| private volatile boolean shouldServiceRun = true; | ||
| private final DataNode dn; | ||
| private final DNConf dnConf; | ||
|
|
@@ -135,6 +142,10 @@ static enum RunningState { | |
| dnConf.getLifelineIntervalMs(), dnConf.blockReportInterval, | ||
| dnConf.outliersReportIntervalMs); | ||
| // get the value of maxDataLength. | ||
| sendIBRLock = new Object(); | ||
| ibrExecutorService = Executors.newSingleThreadExecutor( | ||
| new ThreadFactoryBuilder().setDaemon(true) | ||
| .setNameFormat("ibr-executor-%d").build()); | ||
| this.maxDataLength = dnConf.getMaxDataLength(); | ||
| } | ||
|
|
||
|
|
@@ -349,6 +360,11 @@ private long generateUniqueBlockReportId() { | |
| * @throws IOException | ||
| */ | ||
| List<DatanodeCommand> blockReport(long fullBrLeaseId) throws IOException { | ||
|
|
||
| synchronized (sendIBRLock) { | ||
| reportReceivedDeletedBlocks(); | ||
| } | ||
|
|
||
| final ArrayList<DatanodeCommand> cmds = new ArrayList<DatanodeCommand>(); | ||
|
|
||
| // Flush any block information that precedes the block report. Otherwise | ||
|
|
@@ -569,6 +585,9 @@ void stop() { | |
| if (bpThread != null) { | ||
| bpThread.interrupt(); | ||
| } | ||
| if (ibrExecutorService != null && !ibrExecutorService.isShutdown()) { | ||
| ibrExecutorService.shutdownNow(); | ||
| } | ||
| } | ||
|
|
||
| //This must be called only by blockPoolManager | ||
|
|
@@ -590,6 +609,9 @@ private synchronized void cleanUp() { | |
| IOUtils.cleanup(null, bpNamenode); | ||
| IOUtils.cleanup(null, lifelineSender); | ||
| bpos.shutdownActor(this); | ||
| if (!ibrExecutorService.isShutdown()) { | ||
| ibrExecutorService.shutdownNow(); | ||
| } | ||
| } | ||
|
|
||
| private void handleRollingUpgradeStatus(HeartbeatResponse resp) throws IOException { | ||
|
|
@@ -842,6 +864,8 @@ public void run() { | |
| } | ||
|
|
||
| runningState = RunningState.RUNNING; | ||
| ibrExecutorService.submit(new IBRTaskHandler()); | ||
|
|
||
| if (initialRegistrationComplete != null) { | ||
| initialRegistrationComplete.countDown(); | ||
| } | ||
|
|
@@ -1275,4 +1299,5 @@ public long monotonicNow() { | |
| return Time.monotonicNow(); | ||
| } | ||
| } | ||
|
|
||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Avoid |
||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -243,13 +243,18 @@ ReplicaInfo fetchReplicaInfo(String bpid, long blockId) { | |
| @Override // FsDatasetSpi | ||
| public LengthInputStream getMetaDataInputStream(ExtendedBlock b) | ||
| throws IOException { | ||
| File meta = FsDatasetUtil.getMetaFile(getBlockFile(b), b.getGenerationStamp()); | ||
| FsVolumeSpi volume = null; | ||
| File meta = null; | ||
| try { | ||
| meta = FsDatasetUtil.getMetaFile(getBlockFile(b), b.getGenerationStamp()); | ||
| } catch (IOException e) { | ||
| throw new FileNotFoundException("BlockId " + b.getBlockName() + " is not valid."); | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. How did the old exception look like? |
||
| } | ||
|
|
||
| if (meta == null || !meta.exists()) { | ||
| return null; | ||
| } | ||
|
|
||
| FsVolumeSpi volume = null; | ||
| try (AutoCloseableLock lock = datasetWriteLock.acquire()) { | ||
| final ReplicaInfo replicaInfo = getReplicaInfo(b); | ||
| if (replicaInfo != null) { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -42,6 +42,7 @@ | |
| import org.junit.Before; | ||
| import org.junit.Test; | ||
| import org.mockito.Mockito; | ||
| import org.mockito.exceptions.base.MockitoAssertionError; | ||
|
|
||
| /** | ||
| * Verify that incremental block reports are generated in response to | ||
|
|
@@ -124,7 +125,7 @@ public void testReportBlockReceived() throws InterruptedException, IOException { | |
|
|
||
| // Sleep for a very short time, this is necessary since the IBR is | ||
| // generated asynchronously. | ||
| Thread.sleep(2000); | ||
| Thread.sleep(1000); | ||
|
|
||
| // Ensure that the received block was reported immediately. | ||
| Mockito.verify(nnSpy, times(1)).blockReceivedAndDeleted( | ||
|
|
@@ -166,13 +167,28 @@ public void testReportBlockDeleted() throws InterruptedException, IOException { | |
|
|
||
| // Trigger a heartbeat, this also triggers an IBR. | ||
| DataNodeTestUtils.triggerHeartbeat(singletonDn); | ||
| Thread.sleep(2000); | ||
|
|
||
| // Ensure that the deleted block is reported. | ||
| Mockito.verify(nnSpy, times(1)).blockReceivedAndDeleted( | ||
| any(DatanodeRegistration.class), | ||
| anyString(), | ||
| any(StorageReceivedDeletedBlocks[].class)); | ||
| int retries = 0; | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. LambdaTestUtils#waitFor |
||
| while (true) { | ||
| try { | ||
| Mockito.verify(nnSpy, atLeastOnce()).blockReceivedAndDeleted( | ||
| any(DatanodeRegistration.class), | ||
| anyString(), | ||
| any(StorageReceivedDeletedBlocks[].class)); | ||
| break; | ||
| } catch (MockitoAssertionError e) { | ||
| if (retries > 7) { | ||
| throw e; | ||
| } | ||
| retries++; | ||
| Thread.sleep(2000); | ||
| } | ||
| } | ||
|
|
||
| } finally { | ||
| cluster.shutdown(); | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.