[jira] [Created] (HDFS-15386) ReplicaNotFoundException keeps happening in DN after removing multiple DN's data direcotries

2020-06-03 Thread Toshihiro Suzuki (Jira)
Toshihiro Suzuki created HDFS-15386:
---

 Summary: ReplicaNotFoundException keeps happening in DN after 
removing multiple DN's data direcotries
 Key: HDFS-15386
 URL: https://issues.apache.org/jira/browse/HDFS-15386
 Project: Hadoop HDFS
  Issue Type: Bug
Reporter: Toshihiro Suzuki
Assignee: Toshihiro Suzuki


When removing volumes, we need to invalidate all the blocks in the volumes. In 
the following code (FsDatasetImpl), we keep the blocks that will be invalidate 
in *blkToInvalidate* map.
 However as the key of the map is *bpid* (Block Pool ID), it will be 
overwritten by other removed volumes. As a result, the map will have only the 
blocks of the last volume, and invalidate only them:
{code:java}
for (String bpid : volumeMap.getBlockPoolList()) {
  List blocks = new ArrayList<>();
  for (Iterator it =
volumeMap.replicas(bpid).iterator(); it.hasNext();) {
ReplicaInfo block = it.next();
final StorageLocation blockStorageLocation =
block.getVolume().getStorageLocation();
LOG.trace("checking for block " + block.getBlockId() +
" with storageLocation " + blockStorageLocation);
if (blockStorageLocation.equals(sdLocation)) {
  blocks.add(block);
  it.remove();
}
  }
  blkToInvalidate.put(bpid, blocks);
}
{code}
[https://github.com/apache/hadoop/blob/704409d53bf7ebf717a3c2e988ede80f623bbad3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java#L580-L595]



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-dev-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-dev-h...@hadoop.apache.org



[jira] [Created] (HDFS-15298) FIx the findbugs warnings introduced in HDFS-15217

2020-04-23 Thread Toshihiro Suzuki (Jira)
Toshihiro Suzuki created HDFS-15298:
---

 Summary: FIx the findbugs warnings introduced in HDFS-15217
 Key: HDFS-15298
 URL: https://issues.apache.org/jira/browse/HDFS-15298
 Project: Hadoop HDFS
  Issue Type: Bug
Reporter: Toshihiro Suzuki
Assignee: Toshihiro Suzuki


We need to fIx the findbugs warnings introduced in HDFS-15217:
https://builds.apache.org/job/hadoop-multibranch/job/PR-1954/5/artifact/out/new-findbugs-hadoop-hdfs-project_hadoop-hdfs.html



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-dev-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-dev-h...@hadoop.apache.org



[jira] [Created] (HDFS-15249) ThrottledAsyncChecker is not thread-safe.

2020-03-30 Thread Toshihiro Suzuki (Jira)
Toshihiro Suzuki created HDFS-15249:
---

 Summary: ThrottledAsyncChecker is not thread-safe.
 Key: HDFS-15249
 URL: https://issues.apache.org/jira/browse/HDFS-15249
 Project: Hadoop HDFS
  Issue Type: Bug
Reporter: Toshihiro Suzuki
Assignee: Toshihiro Suzuki


ThrottledAsyncChecker should be thread-safe because it can be used by multiple 
threads when we have multiple namespaces.

*checksInProgress* and *completedChecks* are respectively HashMap and 
WeakHashMap which are not tread-safe. So we need to put them in synchronized 
block whenever we access them.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-dev-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-dev-h...@hadoop.apache.org



[jira] [Resolved] (HDFS-14503) ThrottledAsyncChecker throws NPE during block pool initialization

2020-03-30 Thread Toshihiro Suzuki (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-14503?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Toshihiro Suzuki resolved HDFS-14503.
-
Resolution: Duplicate

> ThrottledAsyncChecker throws NPE during block pool initialization 
> --
>
> Key: HDFS-14503
> URL: https://issues.apache.org/jira/browse/HDFS-14503
> Project: Hadoop HDFS
>  Issue Type: Bug
>Affects Versions: 3.3.0
>Reporter: Yiqun Lin
>Priority: Major
>
> ThrottledAsyncChecker throws NPE during block pool initialization. The error 
> leads the block pool registration failure.
> The exception
> {noformat}
> 2019-05-20 01:02:36,003 WARN org.apache.hadoop.hdfs.server.datanode.DataNode: 
> Unexpected exception in block pool Block pool  (Datanode Uuid 
> x) service to xx.xx.xx.xx/xx.xx.xx.xx
> java.lang.NullPointerException
> at 
> org.apache.hadoop.hdfs.server.datanode.checker.ThrottledAsyncChecker$LastCheckResult.access$000(ThrottledAsyncChecker.java:211)
> at 
> org.apache.hadoop.hdfs.server.datanode.checker.ThrottledAsyncChecker.schedule(ThrottledAsyncChecker.java:129)
> at 
> org.apache.hadoop.hdfs.server.datanode.checker.DatasetVolumeChecker.checkAllVolumes(DatasetVolumeChecker.java:209)
> at 
> org.apache.hadoop.hdfs.server.datanode.DataNode.checkDiskError(DataNode.java:3387)
> at 
> org.apache.hadoop.hdfs.server.datanode.DataNode.initBlockPool(DataNode.java:1508)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPOfferService.verifyAndSetNamespaceInfo(BPOfferService.java:319)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.connectToNNAndHandshake(BPServiceActor.java:272)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.run(BPServiceActor.java:768)
> at java.lang.Thread.run(Thread.java:745)
> {noformat}
> Looks like this error due to {{WeakHashMap}} type map {{completedChecks}} has 
> removed the target entry while we still get that entry. Although we have done 
> a check before we get it, there is still a chance the entry is got as null. 
> We met a corner case for this: A federation mode, two block pools in DN, 
> {{ThrottledAsyncChecker}} schedules two same health checks for same volume.
> {noformat}
> 2019-05-20 01:02:36,000 INFO 
> org.apache.hadoop.hdfs.server.datanode.checker.ThrottledAsyncChecker: 
> Scheduling a check for /hadoop/2/hdfs/data/current
> 2019-05-20 01:02:36,000 INFO 
> org.apache.hadoop.hdfs.server.datanode.checker.ThrottledAsyncChecker: 
> Scheduling a check for /hadoop/2/hdfs/data/current
> {noformat}
> {{completedChecks}} cleans up the entry for one successful check after called 
> {{completedChecks#get}}. However, after this, another check we get the null.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-dev-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-dev-h...@hadoop.apache.org



[jira] [Created] (HDFS-15217) Add more information to longest write/read lock held log

2020-03-10 Thread Toshihiro Suzuki (Jira)
Toshihiro Suzuki created HDFS-15217:
---

 Summary: Add more information to longest write/read lock held log
 Key: HDFS-15217
 URL: https://issues.apache.org/jira/browse/HDFS-15217
 Project: Hadoop HDFS
  Issue Type: Improvement
Reporter: Toshihiro Suzuki
Assignee: Toshihiro Suzuki


Currently, we can see the stack trace in the longest write/read lock held log, 
but sometimes we need more information, for example, a target path of deletion:
{code:java}
2020-03-10 21:51:21,116 [main] INFO  namenode.FSNamesystem 
(FSNamesystemLock.java:writeUnlock(276)) -   Number of suppressed write-lock 
reports: 0
Longest write-lock held at 2020-03-10 21:51:21,107+0900 for 6ms via 
java.lang.Thread.getStackTrace(Thread.java:1559)
org.apache.hadoop.util.StringUtils.getStackTrace(StringUtils.java:1058)
org.apache.hadoop.hdfs.server.namenode.FSNamesystemLock.writeUnlock(FSNamesystemLock.java:257)
org.apache.hadoop.hdfs.server.namenode.FSNamesystemLock.writeUnlock(FSNamesystemLock.java:233)
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.writeUnlock(FSNamesystem.java:1706)
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.delete(FSNamesystem.java:3188)
...
{code}
Adding more information (opName, path, etc.) to the log is useful to 
troubleshoot.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-dev-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-dev-h...@hadoop.apache.org



[jira] [Created] (HDFS-15215) The Timestamp for longest write/read lock held log is wrong

2020-03-10 Thread Toshihiro Suzuki (Jira)
Toshihiro Suzuki created HDFS-15215:
---

 Summary: The Timestamp for longest write/read lock held log is 
wrong
 Key: HDFS-15215
 URL: https://issues.apache.org/jira/browse/HDFS-15215
 Project: Hadoop HDFS
  Issue Type: Bug
Reporter: Toshihiro Suzuki
Assignee: Toshihiro Suzuki


I found the Timestamp for longest write/read lock held log is wrong in trunk:

{code}
2020-03-10 16:01:26,585 [main] INFO  namenode.FSNamesystem 
(FSNamesystemLock.java:writeUnlock(281)) -   Number of suppressed write-lock 
reports: 0
Longest write-lock held at 1970-01-03 07:07:40,841+0900 for 3ms via 
java.lang.Thread.getStackTrace(Thread.java:1559)
...
{code}

Looking at the code, it looks like the timestamp comes from System.nanoTime() 
that returns the current value of the running Java Virtual Machine's 
high-resolution time source and this method can only be used to measure elapsed 
time:
https://docs.oracle.com/javase/8/docs/api/java/lang/System.html#nanoTime--

We need to make the timestamp from System.currentTimeMillis().





--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-dev-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-dev-h...@hadoop.apache.org



[jira] [Created] (HDFS-15018) DataNode doesn't shutdown although the number of failed disks reaches dfs.datanode.failed.volumes.tolerated

2019-11-26 Thread Toshihiro Suzuki (Jira)
Toshihiro Suzuki created HDFS-15018:
---

 Summary: DataNode doesn't shutdown although the number of failed 
disks reaches dfs.datanode.failed.volumes.tolerated
 Key: HDFS-15018
 URL: https://issues.apache.org/jira/browse/HDFS-15018
 Project: Hadoop HDFS
  Issue Type: Bug
  Components: datanode
Affects Versions: 2.7.3
 Environment: HDP-2.6.5
Reporter: Toshihiro Suzuki
 Attachments: thread_dumps.txt

In our case, we set dfs.datanode.failed.volumes.tolerated=0 but a DataNode 
didn't shutdown when a disk in the DataNode host got failed for some reason.

The the following log messages were shown in the DataNode log which indicates 
the DataNode detected the disk failure, but the DataNode didn't shutdown:
{code}
2019-09-17T13:15:43.262-0400 WARN 
org.apache.hadoop.hdfs.server.datanode.DataNode: checkDiskErrorAsync callback 
got 1 failed volumes: [/data2/hdfs/current]
2019-09-17T13:15:43.262-0400 INFO 
org.apache.hadoop.hdfs.server.datanode.BlockScanner: Removing scanner for 
volume /data2/hdfs (StorageID DS-329dec9d-a476-4334-9570-651a7e4d1f44)
2019-09-17T13:15:43.263-0400 INFO 
org.apache.hadoop.hdfs.server.datanode.VolumeScanner: 
VolumeScanner(/data2/hdfs, DS-329dec9d-a476-4334-9570-651a7e4d1f44) exiting.
{code}

Looking at the HDFS code, it looks like when the DataNode detects a disk 
failure, DataNode waits until the volume reference of the disk is released.
https://github.com/hortonworks/hadoop/blob/HDP-2.6.5.0-292-tag/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java#L246

I'm suspecting that the volume reference is not released after the failure 
detection, but not sure the reason.

And we took when the issue was happening. Attaching it to this Jira.

It looks like the following thread is waiting for the volume reference of the 
disk to be released:
{code}
"pool-4-thread-1" #174 daemon prio=5 os_prio=0 tid=0x7f9e7c7bf800 
nid=0x8325 in Object.wait() [0x7f9e629cb000]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)
at java.lang.Object.wait(Native Method)
at 
org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeList.waitVolumeRemoved(FsVolumeList.java:262)
at 
org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeList.handleVolumeFailures(FsVolumeList.java:246)
- locked <0x000670559278> (a java.lang.Object)
at 
org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetImpl.handleVolumeFailures(FsDatasetImpl.java:2178)
at 
org.apache.hadoop.hdfs.server.datanode.DataNode.handleVolumeFailures(DataNode.java:3410)
at 
org.apache.hadoop.hdfs.server.datanode.DataNode.access$100(DataNode.java:248)
at 
org.apache.hadoop.hdfs.server.datanode.DataNode$4.call(DataNode.java:2013)
at 
org.apache.hadoop.hdfs.server.datanode.checker.DatasetVolumeChecker$ResultHandler.invokeCallback(DatasetVolumeChecker.java:394)
at 
org.apache.hadoop.hdfs.server.datanode.checker.DatasetVolumeChecker$ResultHandler.cleanup(DatasetVolumeChecker.java:387)
at 
org.apache.hadoop.hdfs.server.datanode.checker.DatasetVolumeChecker$ResultHandler.onFailure(DatasetVolumeChecker.java:370)
at com.google.common.util.concurrent.Futures$6.run(Futures.java:977)
at 
com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:253)
at 
org.apache.hadoop.hdfs.server.datanode.checker.AbstractFuture.executeListener(AbstractFuture.java:991)
at 
org.apache.hadoop.hdfs.server.datanode.checker.AbstractFuture.complete(AbstractFuture.java:885)
at 
org.apache.hadoop.hdfs.server.datanode.checker.AbstractFuture.setException(AbstractFuture.java:739)
at 
org.apache.hadoop.hdfs.server.datanode.checker.TimeoutFuture$Fire.run(TimeoutFuture.java:137)
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
{code}

We found a similar issue HDFS-13339, but we didn't see any dead lock from the 
thread dump.

Attaching the full thread dumps of the problematic DataNode.




--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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

[jira] [Created] (HDFS-14123) NameNode failover doesn't happen when running fsfreeze for the NameNode dir (dfs.namenode.name.dir)

2018-12-04 Thread Toshihiro Suzuki (JIRA)
Toshihiro Suzuki created HDFS-14123:
---

 Summary: NameNode failover doesn't happen when running fsfreeze 
for the NameNode dir (dfs.namenode.name.dir)
 Key: HDFS-14123
 URL: https://issues.apache.org/jira/browse/HDFS-14123
 Project: Hadoop HDFS
  Issue Type: Bug
  Components: ha
Reporter: Toshihiro Suzuki
Assignee: Toshihiro Suzuki


I ran fsfreeze for the NameNode dir (dfs.namenode.name.dir) in my cluster for 
test purpose, but NameNode failover didn't happen.
{code}
fsfreeze -f /mnt
{code}
/mnt is a separate filesystem partition from /. And the NameNode dir 
"dfs.namenode.name.dir" is /mnt/hadoop/hdfs/namenode.

I checked the source code, and I found monitorHealth RPC from ZKFC doesn't fail 
even if the NameNode dir is frozen. I think that's why the failover doesn't 
happen.

Also if the NameNode dir is frozen, it looks like FSImage.rollEditLog() gets 
stuck like the following, and it keeps holding the write lock of FSNamesystem, 
which causes HDFS service down:
{code}
"IPC Server handler 5 on default port 8020" #53 daemon prio=5 os_prio=0 
tid=0x7f56b96e2000 nid=0x5042 in Object.wait() [0x7f56937bb000]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)
at java.lang.Object.wait(Native Method)
at 
org.apache.hadoop.hdfs.server.namenode.FSEditLogAsync$SyncEdit.logSyncWait(FSEditLogAsync.java:317)
- locked <0xc58ca268> (a 
org.apache.hadoop.hdfs.server.namenode.FSEditLogAsync)
at 
org.apache.hadoop.hdfs.server.namenode.FSEditLogAsync.logSyncAll(FSEditLogAsync.java:147)
at 
org.apache.hadoop.hdfs.server.namenode.FSEditLog.endCurrentLogSegment(FSEditLog.java:1422)
- locked <0xc58ca268> (a 
org.apache.hadoop.hdfs.server.namenode.FSEditLogAsync)
at 
org.apache.hadoop.hdfs.server.namenode.FSEditLog.rollEditLog(FSEditLog.java:1316)
- locked <0xc58ca268> (a 
org.apache.hadoop.hdfs.server.namenode.FSEditLogAsync)
at 
org.apache.hadoop.hdfs.server.namenode.FSImage.rollEditLog(FSImage.java:1322)
at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.rollEditLog(FSNamesystem.java:4740)
at 
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.rollEditLog(NameNodeRpcServer.java:1307)
at 
org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolServerSideTranslatorPB.rollEditLog(NamenodeProtocolServerSideTranslatorPB.java:148)
at 
org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos$NamenodeProtocolService$2.callBlockingMethod(NamenodeProtocolProtos.java:14726)
at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:524)
at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1025)
at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:898)
at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:844)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:422)
at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1876)
at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2727)

   Locked ownable synchronizers:
- <0xc5f4ca10> (a 
java.util.concurrent.locks.ReentrantReadWriteLock$FairSync)
{code}

I believe NameNode failover should happen in this case. One idea is to check if 
the NameNode dir is working when NameNode receives monitorHealth RPC from ZKFC.

I will attach a patch for this idea.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: hdfs-dev-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-dev-h...@hadoop.apache.org



[jira] [Created] (HDFS-13949) Correct the description of dfs.datanode.disk.check.timeout in hdfs-default.xml

2018-10-01 Thread Toshihiro Suzuki (JIRA)
Toshihiro Suzuki created HDFS-13949:
---

 Summary: Correct the description of 
dfs.datanode.disk.check.timeout in hdfs-default.xml
 Key: HDFS-13949
 URL: https://issues.apache.org/jira/browse/HDFS-13949
 Project: Hadoop HDFS
  Issue Type: Bug
  Components: documentation
Reporter: Toshihiro Suzuki


The description of dfs.datanode.disk.check.timeout in hdfs-default.xml is as 
follows:
{code}

  dfs.datanode.disk.check.timeout
  10m
  
Maximum allowed time for a disk check to complete during DataNode
startup. If the check does not complete within this time interval
then the disk is declared as failed. This setting supports
multiple time unit suffixes as described in dfs.heartbeat.interval.
If no suffix is specified then milliseconds is assumed.
  

{code}

I don't think the value of this config is used only during DataNode startup. I 
think it's used whenever checking volumes.
The description is misleading so we need to correct it.




--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: hdfs-dev-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-dev-h...@hadoop.apache.org