You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2021/12/05 14:14:09 UTC

[GitHub] [hbase] Apache9 commented on a change in pull request #3800: HBASE-26347 Support detect and exclude slow DNs in fan-out of WAL

Apache9 commented on a change in pull request #3800:
URL: https://github.com/apache/hbase/pull/3800#discussion_r762566409



##########
File path: hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/monitor/ExcludeDatanodeManager.java
##########
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.io.asyncfs.monitor;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.conf.ConfigurationObserver;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.cache.Cache;
+import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
+
+/**
+ * The class to manage the excluded datanodes of the WALs on the regionserver.
+ */
+@InterfaceAudience.Private
+public class ExcludeDatanodeManager implements ConfigurationObserver {
+  private static final Logger LOG = LoggerFactory.getLogger(ExcludeDatanodeManager.class);
+
+  /**
+   * Configure for the max count the excluded datanodes.
+   */
+  private static final String WAL_MAX_EXCLUDE_SLOW_DATANODE_COUNT_KEY =
+    "hbase.regionserver.async.wal.max.exclude.datanode.count";
+  private static final int DEFAULT_WAL_MAX_EXCLUDE_SLOW_DATANODE_COUNT = 3;
+
+  /**
+   * Configure for the TTL time of the datanodes excluded
+   */
+  private static final String WAL_EXCLUDE_DATANODE_TTL_KEY =
+    "hbase.regionserver.async.wal.exclude.datanode.info.ttl.hour";
+  private static final int DEFAULT_WAL_EXCLUDE_DATANODE_TTL = 6; // 6 hours
+
+  private volatile Cache<DatanodeInfo, Long> excludeDNsCache;

Review comment:
       Why volatile?

##########
File path: hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/monitor/ExcludeDatanodeManager.java
##########
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.io.asyncfs.monitor;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.conf.ConfigurationObserver;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.cache.Cache;
+import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
+
+/**
+ * The class to manage the excluded datanodes of the WALs on the regionserver.
+ */
+@InterfaceAudience.Private
+public class ExcludeDatanodeManager implements ConfigurationObserver {
+  private static final Logger LOG = LoggerFactory.getLogger(ExcludeDatanodeManager.class);
+
+  /**
+   * Configure for the max count the excluded datanodes.
+   */
+  private static final String WAL_MAX_EXCLUDE_SLOW_DATANODE_COUNT_KEY =
+    "hbase.regionserver.async.wal.max.exclude.datanode.count";
+  private static final int DEFAULT_WAL_MAX_EXCLUDE_SLOW_DATANODE_COUNT = 3;
+
+  /**
+   * Configure for the TTL time of the datanodes excluded
+   */
+  private static final String WAL_EXCLUDE_DATANODE_TTL_KEY =
+    "hbase.regionserver.async.wal.exclude.datanode.info.ttl.hour";
+  private static final int DEFAULT_WAL_EXCLUDE_DATANODE_TTL = 6; // 6 hours
+
+  private volatile Cache<DatanodeInfo, Long> excludeDNsCache;
+  private final int maxExcludeDNCount;
+  private final Configuration conf;
+  // This is a map of providerId->StreamSlowMonitor
+  private final Map<String, StreamSlowMonitor> streamSlowMonitors =
+    new ConcurrentHashMap<>(1);
+
+  public ExcludeDatanodeManager(Configuration conf) {
+    this.conf = conf;
+    this.maxExcludeDNCount = conf.getInt(WAL_MAX_EXCLUDE_SLOW_DATANODE_COUNT_KEY,
+      DEFAULT_WAL_MAX_EXCLUDE_SLOW_DATANODE_COUNT);
+    this.excludeDNsCache = CacheBuilder.newBuilder()
+      .expireAfterWrite(this.conf.getLong(WAL_EXCLUDE_DATANODE_TTL_KEY,
+        DEFAULT_WAL_EXCLUDE_DATANODE_TTL),
+        TimeUnit.HOURS)
+      .maximumSize(this.maxExcludeDNCount)
+      .concurrencyLevel(10)

Review comment:
       Why 10?

##########
File path: hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/monitor/ExcludeDatanodeManager.java
##########
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.io.asyncfs.monitor;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.conf.ConfigurationObserver;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.cache.Cache;
+import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
+
+/**
+ * The class to manage the excluded datanodes of the WALs on the regionserver.
+ */
+@InterfaceAudience.Private
+public class ExcludeDatanodeManager implements ConfigurationObserver {
+  private static final Logger LOG = LoggerFactory.getLogger(ExcludeDatanodeManager.class);
+
+  /**
+   * Configure for the max count the excluded datanodes.
+   */
+  private static final String WAL_MAX_EXCLUDE_SLOW_DATANODE_COUNT_KEY =
+    "hbase.regionserver.async.wal.max.exclude.datanode.count";
+  private static final int DEFAULT_WAL_MAX_EXCLUDE_SLOW_DATANODE_COUNT = 3;
+
+  /**
+   * Configure for the TTL time of the datanodes excluded
+   */
+  private static final String WAL_EXCLUDE_DATANODE_TTL_KEY =
+    "hbase.regionserver.async.wal.exclude.datanode.info.ttl.hour";
+  private static final int DEFAULT_WAL_EXCLUDE_DATANODE_TTL = 6; // 6 hours
+
+  private volatile Cache<DatanodeInfo, Long> excludeDNsCache;
+  private final int maxExcludeDNCount;
+  private final Configuration conf;
+  // This is a map of providerId->StreamSlowMonitor
+  private final Map<String, StreamSlowMonitor> streamSlowMonitors =
+    new ConcurrentHashMap<>(1);
+
+  public ExcludeDatanodeManager(Configuration conf) {
+    this.conf = conf;
+    this.maxExcludeDNCount = conf.getInt(WAL_MAX_EXCLUDE_SLOW_DATANODE_COUNT_KEY,
+      DEFAULT_WAL_MAX_EXCLUDE_SLOW_DATANODE_COUNT);
+    this.excludeDNsCache = CacheBuilder.newBuilder()
+      .expireAfterWrite(this.conf.getLong(WAL_EXCLUDE_DATANODE_TTL_KEY,
+        DEFAULT_WAL_EXCLUDE_DATANODE_TTL),
+        TimeUnit.HOURS)
+      .maximumSize(this.maxExcludeDNCount)
+      .concurrencyLevel(10)
+      .build();
+  }
+
+  /**
+   * Try to add a datanode to the regionserver excluding cache
+   * @param datanodeInfo the datanode to be added to the excluded cache
+   * @param cause the cause that the datanode is hope to be excluded
+   * @return True if the datanode is added to the regionserver excluding cache, false otherwise
+   */
+  public boolean tryAddExcludeDN(DatanodeInfo datanodeInfo, String cause) {
+    boolean alreadyMarkedSlow = getExcludeDNs().containsKey(datanodeInfo);
+    if (excludeDNsCache.size() < maxExcludeDNCount) {
+      if (!alreadyMarkedSlow) {
+        excludeDNsCache.put(datanodeInfo, System.currentTimeMillis());

Review comment:
       And guava Cache itself can handle the eviction when exceeding the max size?

##########
File path: hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java
##########
@@ -479,24 +487,26 @@ private static FanOutOneBlockAsyncDFSOutput createOutput(DistributedFileSystem d
       List<Future<Channel>> futureList = null;
       try {
         DataChecksum summer = createChecksum(client);
-        locatedBlock = namenode.addBlock(src, client.getClientName(), null, excludesNodes,
-          stat.getFileId(), null, null);
-        List<Channel> datanodeList = new ArrayList<>();
+        locatedBlock = namenode.addBlock(src, client.getClientName(), null,
+          toExcludeNodes.toArray(new DatanodeInfo[0]), stat.getFileId(), null, null);
+        Map<Channel, DatanodeInfo> datanodes = new HashMap<>();

Review comment:
       Better use IdentityHashMap here? 

##########
File path: hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/monitor/StreamSlowMonitor.java
##########
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.io.asyncfs.monitor;
+
+import java.util.Deque;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedDeque;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.conf.ConfigurationObserver;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Class for monitor the wal file flush performance.
+ * Each active wal file has a StreamSlowMonitor.
+ */
+@InterfaceAudience.Private
+public class StreamSlowMonitor implements ConfigurationObserver {
+  private static final Logger LOG = LoggerFactory.getLogger(StreamSlowMonitor.class);
+
+  /**
+   * Configure for the min count for a datanode detected slow.
+   * If a datanode is detected slow times up to this count, then it will be added to the exclude
+   * datanode cache by {@link ExcludeDatanodeManager#tryAddExcludeDN(DatanodeInfo, String)}
+   * of this regionsever.
+   */
+  private static final String WAL_SLOW_DETECT_MIN_COUNT_KEY =
+    "hbase.regionserver.async.wal.min.slow.detect.count";
+  private static final int DEFAULT_WAL_SLOW_DETECT_MIN_COUNT = 3;
+
+  /**
+   * Configure for the TTL of the data that a datanode detected slow.
+   */
+  private static final String WAL_SLOW_DETECT_DATA_TTL_KEY =
+    "hbase.regionserver.async.wal.slow.detect.data.ttl.ms";
+  private static final long DEFAULT_WAL_SLOW_DETECT_DATA_TTL = 10 * 60 * 1000; // 10min in ms
+
+  /**
+   * Configure for the slow packet process time, a duration from send to ACK.
+   */
+  private static final String DATANODE_SLOW_PACKET_PROCESS_TIME_KEY =
+    "hbase.regionserver.async.wal.datanode.slow.packet.process.time.millis";
+  private static final long DEFAULT_DATANODE_SLOW_PACKET_PROCESS_TIME = 6000; //6s in ms
+
+  /**
+   * Configure for the packet flush speed.
+   */
+  private static final String DATANODE_SLOW_PACKET_FLUSH_SPEED_KEY =
+    "hbase.regionserver.async.wal.datanode.slow.packet.flush.speed.kbs";
+  private static final double DEFAULT_DATANODE_SLOW_PACKET_FLUSH_SPEED = 0.1;
+
+  private final String name;
+  // this is a map of datanodeInfo->queued slow PacketAckData
+  private final Map<DatanodeInfo, Deque<PacketAckData>> datanodeSlowDataQueue =
+    new ConcurrentHashMap<>();
+  private final ExcludeDatanodeManager excludeDatanodeManager;
+
+  private int minSlowDetectCount;
+  private long slowDataTtl;
+  private long slowPacketAckMillis;
+  private double slowPacketAckSpeed;
+
+  public StreamSlowMonitor(Configuration conf, String name,
+      ExcludeDatanodeManager excludeDatanodeManager) {
+    setConf(conf);
+    this.name = name;
+    this.excludeDatanodeManager = excludeDatanodeManager;
+    LOG.info("New stream slow monitor {}", this.name);
+  }
+
+  public static StreamSlowMonitor create(Configuration conf, String name) {
+    return new StreamSlowMonitor(conf, name, new ExcludeDatanodeManager(conf));
+  }
+
+  /**
+   * Check if the packet process time shows that the relevant datanode is a slow node.
+   * @param datanodeInfo the datanode that processed the packet
+   * @param packetDataLen the data length of the packet
+   * @param processTime the process time of the packet on the datanode
+   * @param lastAckTimestamp the last acked timestamp of the packet on another datanode
+   * @param unfinished if the packet is unfinished flushed to the datanode replicas
+   */
+  public void checkProcessTimeAndSpeed(DatanodeInfo datanodeInfo, long packetDataLen,
+      long processTime, long lastAckTimestamp, int unfinished) {
+    long current = EnvironmentEdgeManager.currentTime();
+    boolean slow = processTime > slowPacketAckMillis ||
+        (packetDataLen > 100 && (double) packetDataLen / processTime < slowPacketAckSpeed);
+    if (slow) {
+      // check if large diff ack timestamp between replicas
+      if ((lastAckTimestamp > 0 && current - lastAckTimestamp > slowPacketAckMillis / 2) || (
+          lastAckTimestamp <= 0 && unfinished == 0)) {
+        LOG.info("Slow datanode: {}, data length={}, duration={}ms, unfinishedReplicas={}, "
+            + "lastAckTimestamp={}, monitor name: {}", datanodeInfo, packetDataLen, processTime,
+          unfinished, lastAckTimestamp, this.name);
+        if (addSlowAckData(datanodeInfo, packetDataLen, processTime)) {
+          excludeDatanodeManager.tryAddExcludeDN(datanodeInfo, "slow packet ack");
+        }
+      }
+    }
+  }
+
+  @Override
+  public void onConfigurationChange(Configuration conf) {
+    setConf(conf);
+  }
+
+  private boolean addSlowAckData(DatanodeInfo datanodeInfo, long dataLength, long processTime) {
+    Deque<PacketAckData> slowDNQueue = datanodeSlowDataQueue.computeIfAbsent(datanodeInfo,

Review comment:
       When will we remove entries from this Map and the Deque? So once a datanode is marked as slow, it will always be marked as slow? And finally the map will contain all the datanodes in the cluster?

##########
File path: hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/monitor/StreamSlowMonitor.java
##########
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.io.asyncfs.monitor;
+
+import java.util.Deque;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedDeque;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.conf.ConfigurationObserver;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Class for monitor the wal file flush performance.
+ * Each active wal file has a StreamSlowMonitor.
+ */
+@InterfaceAudience.Private
+public class StreamSlowMonitor implements ConfigurationObserver {
+  private static final Logger LOG = LoggerFactory.getLogger(StreamSlowMonitor.class);
+
+  /**
+   * Configure for the min count for a datanode detected slow.
+   * If a datanode is detected slow times up to this count, then it will be added to the exclude
+   * datanode cache by {@link ExcludeDatanodeManager#tryAddExcludeDN(DatanodeInfo, String)}
+   * of this regionsever.
+   */
+  private static final String WAL_SLOW_DETECT_MIN_COUNT_KEY =
+    "hbase.regionserver.async.wal.min.slow.detect.count";
+  private static final int DEFAULT_WAL_SLOW_DETECT_MIN_COUNT = 3;
+
+  /**
+   * Configure for the TTL of the data that a datanode detected slow.
+   */
+  private static final String WAL_SLOW_DETECT_DATA_TTL_KEY =
+    "hbase.regionserver.async.wal.slow.detect.data.ttl.ms";
+  private static final long DEFAULT_WAL_SLOW_DETECT_DATA_TTL = 10 * 60 * 1000; // 10min in ms
+
+  /**
+   * Configure for the slow packet process time, a duration from send to ACK.
+   */
+  private static final String DATANODE_SLOW_PACKET_PROCESS_TIME_KEY =
+    "hbase.regionserver.async.wal.datanode.slow.packet.process.time.millis";
+  private static final long DEFAULT_DATANODE_SLOW_PACKET_PROCESS_TIME = 6000; //6s in ms
+
+  /**
+   * Configure for the packet flush speed.
+   */
+  private static final String DATANODE_SLOW_PACKET_FLUSH_SPEED_KEY =
+    "hbase.regionserver.async.wal.datanode.slow.packet.flush.speed.kbs";
+  private static final double DEFAULT_DATANODE_SLOW_PACKET_FLUSH_SPEED = 0.1;
+
+  private final String name;
+  // this is a map of datanodeInfo->queued slow PacketAckData
+  private final Map<DatanodeInfo, Deque<PacketAckData>> datanodeSlowDataQueue =
+    new ConcurrentHashMap<>();
+  private final ExcludeDatanodeManager excludeDatanodeManager;
+
+  private int minSlowDetectCount;
+  private long slowDataTtl;
+  private long slowPacketAckMillis;
+  private double slowPacketAckSpeed;
+
+  public StreamSlowMonitor(Configuration conf, String name,
+      ExcludeDatanodeManager excludeDatanodeManager) {
+    setConf(conf);
+    this.name = name;
+    this.excludeDatanodeManager = excludeDatanodeManager;
+    LOG.info("New stream slow monitor {}", this.name);
+  }
+
+  public static StreamSlowMonitor create(Configuration conf, String name) {
+    return new StreamSlowMonitor(conf, name, new ExcludeDatanodeManager(conf));
+  }
+
+  /**
+   * Check if the packet process time shows that the relevant datanode is a slow node.
+   * @param datanodeInfo the datanode that processed the packet
+   * @param packetDataLen the data length of the packet
+   * @param processTime the process time of the packet on the datanode
+   * @param lastAckTimestamp the last acked timestamp of the packet on another datanode
+   * @param unfinished if the packet is unfinished flushed to the datanode replicas
+   */
+  public void checkProcessTimeAndSpeed(DatanodeInfo datanodeInfo, long packetDataLen,
+      long processTime, long lastAckTimestamp, int unfinished) {
+    long current = EnvironmentEdgeManager.currentTime();
+    boolean slow = processTime > slowPacketAckMillis ||
+        (packetDataLen > 100 && (double) packetDataLen / processTime < slowPacketAckSpeed);
+    if (slow) {
+      // check if large diff ack timestamp between replicas
+      if ((lastAckTimestamp > 0 && current - lastAckTimestamp > slowPacketAckMillis / 2) || (
+          lastAckTimestamp <= 0 && unfinished == 0)) {
+        LOG.info("Slow datanode: {}, data length={}, duration={}ms, unfinishedReplicas={}, "
+            + "lastAckTimestamp={}, monitor name: {}", datanodeInfo, packetDataLen, processTime,
+          unfinished, lastAckTimestamp, this.name);
+        if (addSlowAckData(datanodeInfo, packetDataLen, processTime)) {
+          excludeDatanodeManager.tryAddExcludeDN(datanodeInfo, "slow packet ack");
+        }
+      }
+    }
+  }
+
+  @Override
+  public void onConfigurationChange(Configuration conf) {
+    setConf(conf);
+  }
+
+  private boolean addSlowAckData(DatanodeInfo datanodeInfo, long dataLength, long processTime) {
+    Deque<PacketAckData> slowDNQueue = datanodeSlowDataQueue.computeIfAbsent(datanodeInfo,
+      d -> new ConcurrentLinkedDeque<>());
+    long current = System.currentTimeMillis();

Review comment:
       Use EnvironmentEdgeManager.currentTime.

##########
File path: hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/monitor/ExcludeDatanodeManager.java
##########
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.io.asyncfs.monitor;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.conf.ConfigurationObserver;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.cache.Cache;
+import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
+
+/**
+ * The class to manage the excluded datanodes of the WALs on the regionserver.
+ */
+@InterfaceAudience.Private
+public class ExcludeDatanodeManager implements ConfigurationObserver {
+  private static final Logger LOG = LoggerFactory.getLogger(ExcludeDatanodeManager.class);
+
+  /**
+   * Configure for the max count the excluded datanodes.
+   */
+  private static final String WAL_MAX_EXCLUDE_SLOW_DATANODE_COUNT_KEY =
+    "hbase.regionserver.async.wal.max.exclude.datanode.count";
+  private static final int DEFAULT_WAL_MAX_EXCLUDE_SLOW_DATANODE_COUNT = 3;
+
+  /**
+   * Configure for the TTL time of the datanodes excluded
+   */
+  private static final String WAL_EXCLUDE_DATANODE_TTL_KEY =
+    "hbase.regionserver.async.wal.exclude.datanode.info.ttl.hour";
+  private static final int DEFAULT_WAL_EXCLUDE_DATANODE_TTL = 6; // 6 hours
+
+  private volatile Cache<DatanodeInfo, Long> excludeDNsCache;
+  private final int maxExcludeDNCount;
+  private final Configuration conf;
+  // This is a map of providerId->StreamSlowMonitor
+  private final Map<String, StreamSlowMonitor> streamSlowMonitors =
+    new ConcurrentHashMap<>(1);
+
+  public ExcludeDatanodeManager(Configuration conf) {
+    this.conf = conf;
+    this.maxExcludeDNCount = conf.getInt(WAL_MAX_EXCLUDE_SLOW_DATANODE_COUNT_KEY,
+      DEFAULT_WAL_MAX_EXCLUDE_SLOW_DATANODE_COUNT);
+    this.excludeDNsCache = CacheBuilder.newBuilder()
+      .expireAfterWrite(this.conf.getLong(WAL_EXCLUDE_DATANODE_TTL_KEY,
+        DEFAULT_WAL_EXCLUDE_DATANODE_TTL),
+        TimeUnit.HOURS)
+      .maximumSize(this.maxExcludeDNCount)
+      .concurrencyLevel(10)
+      .build();
+  }
+
+  /**
+   * Try to add a datanode to the regionserver excluding cache
+   * @param datanodeInfo the datanode to be added to the excluded cache
+   * @param cause the cause that the datanode is hope to be excluded
+   * @return True if the datanode is added to the regionserver excluding cache, false otherwise
+   */
+  public boolean tryAddExcludeDN(DatanodeInfo datanodeInfo, String cause) {
+    boolean alreadyMarkedSlow = getExcludeDNs().containsKey(datanodeInfo);
+    if (excludeDNsCache.size() < maxExcludeDNCount) {
+      if (!alreadyMarkedSlow) {
+        excludeDNsCache.put(datanodeInfo, System.currentTimeMillis());

Review comment:
       Although Cache is thread safe maybe, but here we could add entries more than maxExcludeDNCount if several threads call this method concurrently?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org