You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-issues@hadoop.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2022/07/16 16:51:00 UTC

[jira] [Work logged] (HDFS-16664) Use correct GenerationStamp when invalidating corrupt block replicas

     [ https://issues.apache.org/jira/browse/HDFS-16664?focusedWorklogId=791667&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-791667 ]

ASF GitHub Bot logged work on HDFS-16664:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 16/Jul/22 16:50
            Start Date: 16/Jul/22 16:50
    Worklog Time Spent: 10m 
      Work Description: KevinWikant opened a new pull request, #4568:
URL: https://github.com/apache/hadoop/pull/4568

   ### Description of PR
   
   Under certain conditions the Namenode can send the incorrect generationStamp to a datanode when invalidating a corrupt block replica.
   
   - the generationStamp sent in the DNA_INVALIDATE is based on the [generationStamp of the block sent in the block report](https://github.com/apache/hadoop/blob/8774f178686487007dcf8c418c989b785a529000/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java#L3733)
   - the problem is that the datanode with the corrupt block replica (that receives the DNA_INVALIDATE) is not necissarily the same datanode that sent the block report
   - this can cause the above exception when the corrupt block replica on the datanode receiving the DNA_INVALIDATE & the block replica on the datanode that sent the block report have different generationStamps
   
   Results in the following datanode exception:
   
   ```
   2022-07-16 08:07:52,041 [BP-958471676-X-1657973243350 heartbeating to localhost/127.0.0.1:61365] WARN  datanode.DataNode (BPServiceActor.java:processCommand(887)) - Error processing datanode Command
   java.io.IOException: Failed to delete 1 (out of 1) replica(s):
   0) Failed to delete replica blk_1073741825_1005: GenerationStamp not matched, existing replica is blk_1073741825_1001
           at org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetImpl.invalidate(FsDatasetImpl.java:2139)
           at org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetImpl.invalidate(FsDatasetImpl.java:2034)
           at org.apache.hadoop.hdfs.server.datanode.BPOfferService.processCommandFromActive(BPOfferService.java:735)
           at org.apache.hadoop.hdfs.server.datanode.BPOfferService.processCommandFromActor(BPOfferService.java:680)
           at org.apache.hadoop.hdfs.server.datanode.BPServiceActor.processCommand(BPServiceActor.java:883)
           at org.apache.hadoop.hdfs.server.datanode.BPServiceActor.offerService(BPServiceActor.java:678)
           at org.apache.hadoop.hdfs.server.datanode.BPServiceActor.run(BPServiceActor.java:849)
           at java.lang.Thread.run(Thread.java:750)
   ```
   
   ### How was this patch tested?
   
   Validated the fix by leveraging the unit test "TestDecommission#testDeleteCorruptReplicaForUnderReplicatedBlock"
   
   ## Failed Test - Before this change
   
   ```
   > mvn test -Dtest=TestDecommission#testDeleteCorruptReplicaForUnderReplicatedBlock
   
   
   [INFO] Results:
   [INFO] 
   [ERROR] Failures: 
   [ERROR]   TestDecommission.testDeleteCorruptReplicaForUnderReplicatedBlock:2035 Node 127.0.0.1:61366 failed to complete decommissioning. numTrackedNodes=1 , numPendingNodes=0 , adminState=Decommission In Progress , nodesWithReplica=[127.0.0.1:61366, 127.0.0.1:61419]
   
   ```
   
   ```
   > cat target/surefire-reports/org.apache.hadoop.hdfs.TestDecommission-output.txt | grep 'Expected Replicas:\|XXX\|FINALIZED\|Block now\|Failed to delete'
   
   
   2022-07-16 08:07:45,891 [Listener at localhost/61378] INFO  hdfs.TestDecommission (TestDecommission.java:testDeleteCorruptReplicaForUnderReplicatedBlock(1942)) - Block now has 2 corrupt replicas on [127.0.0.1:61370 , 127.0.0.1:61375] and 1 live replica on 127.0.0.1:61366
   2022-07-16 08:07:45,913 [Listener at localhost/61378] INFO  hdfs.TestDecommission (TestDecommission.java:testDeleteCorruptReplicaForUnderReplicatedBlock(1974)) - Block now has 2 corrupt replicas on [127.0.0.1:61370 , 127.0.0.1:61375] and 1 decommissioning replica on 127.0.0.1:61366
   XXX invalidateBlock dn=127.0.0.1:61415 , blk=1073741825_1001
   XXX postponeBlock dn=127.0.0.1:61415 , blk=1073741825_1001
   XXX invalidateBlock dn=127.0.0.1:61419 , blk=1073741825_1003
   XXX addToInvalidates dn=127.0.0.1:61419 , blk=1073741825_1003
   XXX addBlocksToBeInvalidated dn=127.0.0.1:61419 , blk=1073741825_1003
   XXX rescanPostponedMisreplicatedBlocks blk=1073741825_1005
   XXX DNA_INVALIDATE dn=/127.0.0.1:61419 , blk=1073741825_1003
   XXX invalidate(on DN) dn=/127.0.0.1:61419 , invalidBlk=blk_1073741825_1003 , blkByIdAndGenStamp = FinalizedReplica, blk_1073741825_1003, FINALIZED
   2022-07-16 08:07:49,084 [BP-958471676-X-1657973243350 heartbeating to localhost/127.0.0.1:61365] INFO  impl.FsDatasetAsyncDiskService (FsDatasetAsyncDiskService.java:deleteAsync(226)) - Scheduling blk_1073741825_1003 replica FinalizedReplica, blk_1073741825_1003, FINALIZED
   XXX addBlock dn=127.0.0.1:61419 , blk=1073741825_1005   <<<  block report is coming from 127.0.0.1:61419 which has genStamp=1005
   XXX invalidateCorruptReplicas dn=127.0.0.1:61415 , reported_blk=1073741825_1005   <<<  corrupt replica is on 127.0.0.1:61415 which is expecting genStamp=1001
   XXX addToInvalidates dn=127.0.0.1:61415 , blk=1073741825_1005
   2022-07-16 08:07:49,431 [DatanodeAdminMonitor-0] INFO  BlockStateChange (DatanodeAdminManager.java:logBlockReplicationInfo(417)) - Block: blk_1073741825_1005, Expected Replicas: 2, live replicas: 1, corrupt replicas: 0, decommissioned replicas: 0, decommissioning replicas: 1, maintenance replicas: 0, live entering maintenance replicas: 0, excess replicas: 0, Is Open File: false, Datanodes having this block: 127.0.0.1:61366 127.0.0.1:61419 , Current Datanode: 127.0.0.1:61366, Is current datanode decommissioning: true, Is current datanode entering maintenance: false
   XXX addBlocksToBeInvalidated dn=127.0.0.1:61415 , blk=1073741825_1005   <<<  Namenode sends wrong genStamp to 127.0.0.1:61415
   XXX DNA_INVALIDATE dn=/127.0.0.1:61415 , blk=1073741825_1005
   XXX invalidate(on DN) dn=/127.0.0.1:61415 , invalidBlk=blk_1073741825_1005 , blkByIdAndGenStamp = null
   XXX invalidate(on DN) dn=/127.0.0.1:61415 , invalidBlk=blk_1073741825_1005 , blkById = FinalizedReplica, blk_1073741825_1001, FINALIZED
   2022-07-16 08:07:52,041 [BP-958471676-X-1657973243350 heartbeating to localhost/127.0.0.1:61365] WARN  datanode.DataNode (BPServiceActor.java:processCommand(887)) - Error processing datanode Command
   java.io.IOException: Failed to delete 1 (out of 1) replica(s):
   0) Failed to delete replica blk_1073741825_1005: GenerationStamp not matched, existing replica is blk_1073741825_1001
           at org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetImpl.invalidate(FsDatasetImpl.java:2139)
           at org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetImpl.invalidate(FsDatasetImpl.java:2034)
           at org.apache.hadoop.hdfs.server.datanode.BPOfferService.processCommandFromActive(BPOfferService.java:735)
           at org.apache.hadoop.hdfs.server.datanode.BPOfferService.processCommandFromActor(BPOfferService.java:680)
           at org.apache.hadoop.hdfs.server.datanode.BPServiceActor.processCommand(BPServiceActor.java:883)
           at org.apache.hadoop.hdfs.server.datanode.BPServiceActor.offerService(BPServiceActor.java:678)
           at org.apache.hadoop.hdfs.server.datanode.BPServiceActor.run(BPServiceActor.java:849)
           at java.lang.Thread.run(Thread.java:750)
   2022-07-16 08:07:52,384 [DataXceiver for client  at /127.0.0.1:61434 [Receiving block BP-958471676-X-1657973243350:blk_1073741825_1005]] INFO  datanode.DataNode (DataXceiver.java:writeBlock(939)) - opWriteBlock BP-958471676-X-1657973243350:blk_1073741825_1005 received exception org.apache.hadoop.hdfs.server.datanode.ReplicaAlreadyExistsException: Block BP-958471676-X-1657973243350:blk_1073741825_1005 already exists in state FINALIZED and thus cannot be created.
   2022-07-16 08:07:52,385 [DataXceiver for client  at /127.0.0.1:61434 [Receiving block BP-958471676-X-1657973243350:blk_1073741825_1005]] INFO  datanode.DataNode (DataXceiver.java:run(307)) - 127.0.0.1:61415:DataXceiver error processing WRITE_BLOCK operation  src: /127.0.0.1:61434 dst: /127.0.0.1:61415; org.apache.hadoop.hdfs.server.datanode.ReplicaAlreadyExistsException: Block BP-958471676-X-1657973243350:blk_1073741825_1005 already exists in state FINALIZED and thus cannot be created.
   2022-07-16 08:07:54,422 [DatanodeAdminMonitor-0] INFO  BlockStateChange (DatanodeAdminManager.java:logBlockReplicationInfo(417)) - Block: blk_1073741825_1005, Expected Replicas: 2, live replicas: 1, corrupt replicas: 0, decommissioned replicas: 0, decommissioning replicas: 1, maintenance replicas: 0, live entering maintenance replicas: 0, excess replicas: 0, Is Open File: false, Datanodes having this block: 127.0.0.1:61366 127.0.0.1:61419 , Current Datanode: 127.0.0.1:61366, Is current datanode decommissioning: true, Is current datanode entering maintenance: false
   ...
   2022-07-16 08:08:24,426 [DatanodeAdminMonitor-0] INFO  BlockStateChange (DatanodeAdminManager.java:logBlockReplicationInfo(417)) - Block: blk_1073741825_1005, Expected Replicas: 2, live replicas: 1, corrupt replicas: 0, decommissioned replicas: 0, decommissioning replicas: 1, maintenance replicas: 0, live entering maintenance replicas: 0, excess replicas: 0, Is Open File: false, Datanodes having this block: 127.0.0.1:61366 127.0.0.1:61419 , Current Datanode: 127.0.0.1:61366, Is current datanode decommissioning: true, Is current datanode entering maintenance: false
   ```
   
   Note the inline comments above which illustrate the bug
   
   
   ## Successful Test - After this change
   
   ```
   > mvn test -Dtest=TestDecommission#testDeleteCorruptReplicaForUnderReplicatedBlock
   
   
   [INFO] Results:
   [INFO] 
   [INFO] Tests run: 1, Failures: 0, Errors: 0, Skipped: 0
   ```
   
   Logs:
   
   ```
   > cat target/surefire-reports/org.apache.hadoop.hdfs.TestDecommission-output.txt | grep 'Expected Replicas:\|XXX\|FINALIZED\|Block now\|Failed to delete'
   
   
   2022-07-16 07:54:30,648 [Listener at localhost/60376] INFO  hdfs.TestDecommission (TestDecommission.java:testDeleteCorruptReplicaForUnderReplicatedBlock(1942)) - Block now has 2 corrupt replicas on [127.0.0.1:60364 , 127.0.0.1:60368] and 1 live replica on 127.0.0.1:60373
   2022-07-16 07:54:30,669 [Listener at localhost/60376] INFO  hdfs.TestDecommission (TestDecommission.java:testDeleteCorruptReplicaForUnderReplicatedBlock(1974)) - Block now has 2 corrupt replicas on [127.0.0.1:60364 , 127.0.0.1:60368] and 1 decommissioning replica on 127.0.0.1:60373
   XXX invalidateBlock dn=127.0.0.1:60423 , blk=1073741825_1001
   XXX postponeBlock dn=127.0.0.1:60423 , blk=1073741825_1001
   XXX invalidateBlock dn=127.0.0.1:60427 , blk=1073741825_1003
   XXX addToInvalidates dn=127.0.0.1:60427 , blk=1073741825_1003
   XXX addBlocksToBeInvalidated dn=127.0.0.1:60427 , blk=1073741825_1003
   XXX rescanPostponedMisreplicatedBlocks blk=1073741825_1005
   XXX DNA_INVALIDATE dn=/127.0.0.1:60427 , blk=1073741825_1003
   XXX invalidate(on DN) dn=/127.0.0.1:60427 , invalidBlk=blk_1073741825_1003 , blkByIdAndGenStamp = FinalizedReplica, blk_1073741825_1003, FINALIZED
   2022-07-16 07:54:32,831 [BP-1469857843-X-1657972447604 heartbeating to localhost/127.0.0.1:60363] INFO  impl.FsDatasetAsyncDiskService (FsDatasetAsyncDiskService.java:deleteAsync(226)) - Scheduling blk_1073741825_1003 replica FinalizedReplica, blk_1073741825_1003, FINALIZED
   2022-07-16 07:54:33,772 [DatanodeAdminMonitor-0] INFO  BlockStateChange (DatanodeAdminManager.java:logBlockReplicationInfo(417)) - Block: blk_1073741825_1005, Expected Replicas: 2, live replicas: 0, corrupt replicas: 1, decommissioned replicas: 0, decommissioning replicas: 1, maintenance replicas: 0, live entering maintenance replicas: 0, excess replicas: 0, Is Open File: false, Datanodes having this block: 127.0.0.1:60373 127.0.0.1:60423 , Current Datanode: 127.0.0.1:60373, Is current datanode decommissioning: true, Is current datanode entering maintenance: false
   XXX addBlock dn=127.0.0.1:60427 , blk=1073741825_1005
   XXX invalidateCorruptReplicas dn=127.0.0.1:60423 , reported_blk=1073741825_1005
   XXX getCorruptReplicaGenerationStamp dn=127.0.0.1:60423 , genStamp=1001
   XXX addToInvalidates dn=127.0.0.1:60423 , blk=1073741825_1001
   XXX addBlocksToBeInvalidated dn=127.0.0.1:60423 , blk=1073741825_1001
   XXX DNA_INVALIDATE dn=/127.0.0.1:60423 , blk=1073741825_1001
   XXX invalidate(on DN) dn=/127.0.0.1:60423 , invalidBlk=blk_1073741825_1001 , blkByIdAndGenStamp = FinalizedReplica, blk_1073741825_1001, FINALIZED
   2022-07-16 07:54:35,796 [BP-1469857843-X-1657972447604 heartbeating to localhost/127.0.0.1:60363] INFO  impl.FsDatasetAsyncDiskService (FsDatasetAsyncDiskService.java:deleteAsync(226)) - Scheduling blk_1073741825_1001 replica FinalizedReplica, blk_1073741825_1001, FINALIZED
   XXX addBlock dn=127.0.0.1:60423 , blk=1073741825_1005
   2022-07-16 07:54:40,768 [Listener at localhost/60430] INFO  hdfs.TestDecommission (TestDecommission.java:testDeleteCorruptReplicaForUnderReplicatedBlock(2050)) - Block now has 2 live replicas on [127.0.0.1:60423 , 127.0.0.1:60427] and 1 decommissioned replica on 127.0.0.1:60373
   ```
   
   Using "getCorruptReplicaGenerationStamp" allows the Namenode to get the correct generationStamp for the corrupt block replica
   
   ### For code changes:
   
   - [X] Does the title or this PR starts with the corresponding JIRA issue id (e.g. 'HADOOP-17799. Your PR title ...')?
   - [n/a] Object storage: have the integration tests been executed and the endpoint declared according to the connector-specific documentation?
   - [n/a] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   - [n/a] If applicable, have you updated the `LICENSE`, `LICENSE-binary`, `NOTICE-binary` files?
   
   




Issue Time Tracking
-------------------

            Worklog Id:     (was: 791667)
    Remaining Estimate: 0h
            Time Spent: 10m

> Use correct GenerationStamp when invalidating corrupt block replicas
> --------------------------------------------------------------------
>
>                 Key: HDFS-16664
>                 URL: https://issues.apache.org/jira/browse/HDFS-16664
>             Project: Hadoop HDFS
>          Issue Type: Bug
>            Reporter: Kevin Wikant
>            Priority: Major
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
> While trying to backport HDFS-16064 to an older Hadoop version, the new unit test "testDeleteCorruptReplicaForUnderReplicatedBlock" started failing unexpectedly.
> Upon deep diving this unit test failure, I identified a bug in HDFS corrupt replica invalidation which results in the following datanode exception:
> {quote}2022-07-16 08:07:52,041 [BP-958471676-X-1657973243350 heartbeating to localhost/127.0.0.1:61365] WARN  datanode.DataNode (BPServiceActor.java:processCommand(887)) - Error processing datanode Command
> java.io.IOException: Failed to delete 1 (out of 1) replica(s):
> 0) Failed to delete replica blk_1073741825_1005: GenerationStamp not matched, existing replica is blk_1073741825_1001
>         at org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetImpl.invalidate(FsDatasetImpl.java:2139)
>         at org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetImpl.invalidate(FsDatasetImpl.java:2034)
>         at org.apache.hadoop.hdfs.server.datanode.BPOfferService.processCommandFromActive(BPOfferService.java:735)
>         at org.apache.hadoop.hdfs.server.datanode.BPOfferService.processCommandFromActor(BPOfferService.java:680)
>         at org.apache.hadoop.hdfs.server.datanode.BPServiceActor.processCommand(BPServiceActor.java:883)
>         at org.apache.hadoop.hdfs.server.datanode.BPServiceActor.offerService(BPServiceActor.java:678)
>         at org.apache.hadoop.hdfs.server.datanode.BPServiceActor.run(BPServiceActor.java:849)
>         at java.lang.Thread.run(Thread.java:750)
> {quote}
> The issue is that the Namenode is sending wrong generationStamp to the datanode. By adding some additional logs, I was able to determine the root cause for this:
>  * the generationStamp sent in the DNA_INVALIDATE is based on the [generationStamp of the block sent in the block report|https://github.com/apache/hadoop/blob/8774f178686487007dcf8c418c989b785a529000/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java#L3733]
>  * the problem is that the datanode with the corrupt block replica (that receives the DNA_INVALIDATE) is not necissarily the same datanode that sent the block report
>  * this can cause the above exception when the corrupt block replica on the datanode receiving the DNA_INVALIDATE & the block replica on the datanode that sent the block report have different generationStamps
> The solution is to store the corrupt replicas generationStamp in the CorruptReplicasMap, then to extract this correct generationStamp value when sending the DNA_INVALIDATE to the datanode
>  
> h2. Failed Test - Before the fix
> {quote}> mvn test -Dtest=TestDecommission#testDeleteCorruptReplicaForUnderReplicatedBlock
>  
> [INFO] Results:
> [INFO] 
> [ERROR] Failures: 
> [ERROR]   TestDecommission.testDeleteCorruptReplicaForUnderReplicatedBlock:2035 Node 127.0.0.1:61366 failed to complete decommissioning. numTrackedNodes=1 , numPendingNodes=0 , adminState=Decommission In Progress , nodesWithReplica=[127.0.0.1:61366, 127.0.0.1:61419]
> {quote}
> Logs:
> {quote}> cat target/surefire-reports/org.apache.hadoop.hdfs.TestDecommission-output.txt | grep 'Expected Replicas:|XXX|FINALIZED|Block now|Failed to delete'
> 2022-07-16 08:07:45,891 [Listener at localhost/61378] INFO  hdfs.TestDecommission (TestDecommission.java:testDeleteCorruptReplicaForUnderReplicatedBlock(1942)) - Block now has 2 corrupt replicas on [127.0.0.1:61370 , 127.0.0.1:61375] and 1 live replica on 127.0.0.1:61366
> 2022-07-16 08:07:45,913 [Listener at localhost/61378] INFO  hdfs.TestDecommission (TestDecommission.java:testDeleteCorruptReplicaForUnderReplicatedBlock(1974)) - Block now has 2 corrupt replicas on [127.0.0.1:61370 , 127.0.0.1:61375] and 1 decommissioning replica on 127.0.0.1:61366
> XXX invalidateBlock dn=127.0.0.1:61415 , blk=1073741825_1001
> XXX postponeBlock dn=127.0.0.1:61415 , blk=1073741825_1001
> XXX invalidateBlock dn=127.0.0.1:61419 , blk=1073741825_1003
> XXX addToInvalidates dn=127.0.0.1:61419 , blk=1073741825_1003
> XXX addBlocksToBeInvalidated dn=127.0.0.1:61419 , blk=1073741825_1003
> XXX rescanPostponedMisreplicatedBlocks blk=1073741825_1005
> XXX DNA_INVALIDATE dn=/127.0.0.1:61419 , blk=1073741825_1003
> XXX invalidate(on DN) dn=/127.0.0.1:61419 , invalidBlk=blk_1073741825_1003 , blkByIdAndGenStamp = FinalizedReplica, blk_1073741825_1003, FINALIZED
> 2022-07-16 08:07:49,084 [BP-958471676-X-1657973243350 heartbeating to localhost/127.0.0.1:61365] INFO  impl.FsDatasetAsyncDiskService (FsDatasetAsyncDiskService.java:deleteAsync(226)) - Scheduling blk_1073741825_1003 replica FinalizedReplica, blk_1073741825_1003, FINALIZED
> XXX addBlock dn=127.0.0.1:61419 , blk=1073741825_1005   *<<<  block report is coming from 127.0.0.1:61419 which has genStamp=1005*
> XXX invalidateCorruptReplicas dn=127.0.0.1:61415 , reported_blk=1073741825_1005   *<<<  corrupt replica is on 127.0.0.1:61415 which is expecting genStamp=1001*
> XXX addToInvalidates dn=127.0.0.1:61415 , blk=1073741825_1005
> 2022-07-16 08:07:49,431 [DatanodeAdminMonitor-0] INFO  BlockStateChange (DatanodeAdminManager.java:logBlockReplicationInfo(417)) - Block: blk_1073741825_1005, Expected Replicas: 2, live replicas: 1, corrupt replicas: 0, decommissioned replicas: 0, decommissioning replicas: 1, maintenance replicas: 0, live entering maintenance replicas: 0, excess replicas: 0, Is Open File: false, Datanodes having this block: 127.0.0.1:61366 127.0.0.1:61419 , Current Datanode: 127.0.0.1:61366, Is current datanode decommissioning: true, Is current datanode entering maintenance: false
> XXX addBlocksToBeInvalidated dn=127.0.0.1:61415 , blk=1073741825_1005   *<<<  Namenode sends wrong genStamp to 127.0.0.1:61415*
> XXX DNA_INVALIDATE dn=/127.0.0.1:61415 , blk=1073741825_1005
> XXX invalidate(on DN) dn=/127.0.0.1:61415 , invalidBlk=blk_1073741825_1005 , blkByIdAndGenStamp = null
> XXX invalidate(on DN) dn=/127.0.0.1:61415 , invalidBlk=blk_1073741825_1005 , blkById = FinalizedReplica, blk_1073741825_1001, FINALIZED
> 2022-07-16 08:07:52,041 [BP-958471676-X-1657973243350 heartbeating to localhost/127.0.0.1:61365] WARN  datanode.DataNode (BPServiceActor.java:processCommand(887)) - Error processing datanode Command
> java.io.IOException: Failed to delete 1 (out of 1) replica(s):
> 0) Failed to delete replica blk_1073741825_1005: GenerationStamp not matched, existing replica is blk_1073741825_1001
>         at org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetImpl.invalidate(FsDatasetImpl.java:2139)
>         at org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetImpl.invalidate(FsDatasetImpl.java:2034)
>         at org.apache.hadoop.hdfs.server.datanode.BPOfferService.processCommandFromActive(BPOfferService.java:735)
>         at org.apache.hadoop.hdfs.server.datanode.BPOfferService.processCommandFromActor(BPOfferService.java:680)
>         at org.apache.hadoop.hdfs.server.datanode.BPServiceActor.processCommand(BPServiceActor.java:883)
>         at org.apache.hadoop.hdfs.server.datanode.BPServiceActor.offerService(BPServiceActor.java:678)
>         at org.apache.hadoop.hdfs.server.datanode.BPServiceActor.run(BPServiceActor.java:849)
>         at java.lang.Thread.run(Thread.java:750)
> 2022-07-16 08:07:52,384 [DataXceiver for client  at /127.0.0.1:61434 [Receiving block BP-958471676-X-1657973243350:blk_1073741825_1005]] INFO  datanode.DataNode (DataXceiver.java:writeBlock(939)) - opWriteBlock BP-958471676-X-1657973243350:blk_1073741825_1005 received exception org.apache.hadoop.hdfs.server.datanode.ReplicaAlreadyExistsException: Block BP-958471676-X-1657973243350:blk_1073741825_1005 already exists in state FINALIZED and thus cannot be created.
> 2022-07-16 08:07:52,385 [DataXceiver for client  at /127.0.0.1:61434 [Receiving block BP-958471676-X-1657973243350:blk_1073741825_1005]] INFO  datanode.DataNode (DataXceiver.java:run(307)) - 127.0.0.1:61415:DataXceiver error processing WRITE_BLOCK operation  src: /127.0.0.1:61434 dst: /127.0.0.1:61415; org.apache.hadoop.hdfs.server.datanode.ReplicaAlreadyExistsException: Block BP-958471676-X-1657973243350:blk_1073741825_1005 already exists in state FINALIZED and thus cannot be created.
> 2022-07-16 08:07:54,422 [DatanodeAdminMonitor-0] INFO  BlockStateChange (DatanodeAdminManager.java:logBlockReplicationInfo(417)) - Block: blk_1073741825_1005, Expected Replicas: 2, live replicas: 1, corrupt replicas: 0, decommissioned replicas: 0, decommissioning replicas: 1, maintenance replicas: 0, live entering maintenance replicas: 0, excess replicas: 0, Is Open File: false, Datanodes having this block: 127.0.0.1:61366 127.0.0.1:61419 , Current Datanode: 127.0.0.1:61366, Is current datanode decommissioning: true, Is current datanode entering maintenance: false
> ...
> 2022-07-16 08:08:24,426 [DatanodeAdminMonitor-0] INFO  BlockStateChange (DatanodeAdminManager.java:logBlockReplicationInfo(417)) - Block: blk_1073741825_1005, Expected Replicas: 2, live replicas: 1, corrupt replicas: 0, decommissioned replicas: 0, decommissioning replicas: 1, maintenance replicas: 0, live entering maintenance replicas: 0, excess replicas: 0, Is Open File: false, Datanodes having this block: 127.0.0.1:61366 127.0.0.1:61419 , Current Datanode: 127.0.0.1:61366, Is current datanode decommissioning: true, Is current datanode entering maintenance: false
> {quote}
> Note the inline comments above which illustrate the bug
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: hdfs-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-help@hadoop.apache.org