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/10/27 13:58:21 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_r737485574



##########
File path: hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java
##########
@@ -451,15 +454,29 @@ public NameNodeException(Throwable cause) {
 
   private static FanOutOneBlockAsyncDFSOutput createOutput(DistributedFileSystem dfs, String src,
       boolean overwrite, boolean createParent, short replication, long blockSize,
-      EventLoopGroup eventLoopGroup, Class<? extends Channel> channelClass) throws IOException {
+      EventLoopGroup eventLoopGroup, Class<? extends Channel> channelClass,
+      StreamSlowMonitor monitor) throws IOException {
     Configuration conf = dfs.getConf();
     DFSClient client = dfs.getClient();
     String clientName = client.getClientName();
     ClientProtocol namenode = client.getNamenode();
     int createMaxRetries = conf.getInt(ASYNC_DFS_OUTPUT_CREATE_MAX_RETRIES,
       DEFAULT_ASYNC_DFS_OUTPUT_CREATE_MAX_RETRIES);
-    DatanodeInfo[] excludesNodes = EMPTY_DN_ARRAY;
+    ExcludeDatanodeManager excludeDatanodeManager = monitor == null ? null :
+      monitor.getExcludeDatanodeManager();
+    Set<DatanodeInfo> toExcludeNodes = new HashSet<>();
     for (int retry = 0;; retry++) {
+      if (excludeDatanodeManager != null) {
+        toExcludeNodes.addAll(excludeDatanodeManager.getExcludeDNs().keySet());
+      }
+      if (excludeDatanodeManager != null && retry > 1 && retry >= createMaxRetries - 1) {
+        // invalid the exclude cache, to avoid not enough replicas

Review comment:
       I do not think we can make sure that the failure is because of invalid exclude cache here?

##########
File path: hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java
##########
@@ -136,17 +138,22 @@
 
     // should be backed by a thread safe collection
     private final Set<ChannelId> unfinishedReplicas;
+    private final long dataLength;
+    private final long flushTimestamp;
+    private long lastAckTimestamp = -1;
 
     public Callback(CompletableFuture<Long> future, long ackedLength,
-        Collection<Channel> replicas) {
+        final Map<Channel, DatanodeInfo> replicas, long dataLength) {

Review comment:
       Do we really need to pass a Map here? Seems we do not use the value field.

##########
File path: hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/monitor/ExcludeDatanodeManager.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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 java.util.concurrent.locks.ReentrantReadWriteLock;
+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;
+
+@InterfaceAudience.Private
+public class ExcludeDatanodeManager implements ConfigurationObserver {
+  private static final Logger LOG = LoggerFactory.getLogger(ExcludeDatanodeManager.class);
+
+  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;
+
+  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 Cache<DatanodeInfo, Long> excludeDNsCache;
+  private final int maxExcludeDNCount;
+  private final Configuration conf;
+  private final Map<String, StreamSlowMonitor> streamSlowMonitors =
+    new ConcurrentHashMap<>(1);
+  private ReentrantReadWriteLock cacheLock = new ReentrantReadWriteLock();
+
+  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();
+  }
+
+  public Map<DatanodeInfo, Long> getExcludeDNs() {
+    cacheLock.readLock().lock();
+    try {
+      return Collections.unmodifiableMap(excludeDNsCache.asMap());
+    } finally {
+      cacheLock.readLock().unlock();
+    }
+  }
+
+  public boolean addExcludeDN(DatanodeInfo datanodeInfo, String cause) {
+    cacheLock.readLock().lock();
+    try {
+      boolean alreadyMarkedSlow = getExcludeDNs().containsKey(datanodeInfo);
+      if (excludeDNsCache.size() < maxExcludeDNCount) {
+        if (!alreadyMarkedSlow) {
+          excludeDNsCache.put(datanodeInfo, System.currentTimeMillis());
+          LOG.info(
+            "Added datanode: {} to exclude cache by [{}] success, current excludeDNsCache size={}",
+            datanodeInfo, cause, excludeDNsCache.size());
+          return true;
+        }
+      } else {
+        LOG.warn("Try add datanode {} to exclude cache by [{}] failed, up to max exclude limit {}, "
+            + "current exclude DNs are {}", datanodeInfo, cause, excludeDNsCache.size(),
+          getExcludeDNs().keySet());
+      }
+      return false;
+    } finally {
+      cacheLock.readLock().unlock();
+    }
+  }
+
+  public void invalidAllExcludeDNs() {
+    cacheLock.readLock().lock();
+    try {
+      excludeDNsCache.invalidateAll();
+    } finally {
+      cacheLock.readLock().unlock();
+    }
+  }
+
+  public StreamSlowMonitor getStreamSlowMonitor(String name) {
+    String key = name == null || name.isEmpty() ? "defaultMonitorName" : name;
+    return streamSlowMonitors
+      .computeIfAbsent(key, k -> new StreamSlowMonitor(conf, key, this));
+  }
+
+  @Override
+  public void onConfigurationChange(Configuration conf) {
+    for (StreamSlowMonitor monitor : streamSlowMonitors.values()) {
+      monitor.onConfigurationChange(conf);
+    }
+    cacheLock.writeLock().lock();

Review comment:
       OK, so we have extra logic when adding and also here we need to invlidateAll...

##########
File path: hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java
##########
@@ -136,17 +138,22 @@
 
     // should be backed by a thread safe collection
     private final Set<ChannelId> unfinishedReplicas;
+    private final long dataLength;
+    private final long flushTimestamp;
+    private long lastAckTimestamp = -1;
 
     public Callback(CompletableFuture<Long> future, long ackedLength,
-        Collection<Channel> replicas) {
+        final Map<Channel, DatanodeInfo> replicas, long dataLength) {

Review comment:
       And what is dataLength mean here? The name is too general...

##########
File path: hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/monitor/PacketAckData.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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 org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class PacketAckData {

Review comment:
       If it is only used in StreamSlowMonitor, let's make it a nested class? StreamSlowMonitor is not very big so increasing the number of lines is not a big deal.

##########
File path: hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/monitor/ExcludeDatanodeManager.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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 java.util.concurrent.locks.ReentrantReadWriteLock;
+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;
+
+@InterfaceAudience.Private
+public class ExcludeDatanodeManager implements ConfigurationObserver {
+  private static final Logger LOG = LoggerFactory.getLogger(ExcludeDatanodeManager.class);
+
+  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;
+
+  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 Cache<DatanodeInfo, Long> excludeDNsCache;
+  private final int maxExcludeDNCount;
+  private final Configuration conf;
+  private final Map<String, StreamSlowMonitor> streamSlowMonitors =
+    new ConcurrentHashMap<>(1);
+  private ReentrantReadWriteLock cacheLock = new ReentrantReadWriteLock();
+
+  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();
+  }
+
+  public Map<DatanodeInfo, Long> getExcludeDNs() {
+    cacheLock.readLock().lock();

Review comment:
       Guava cache is thead safe IIRC? Do we still need to lock here?




-- 
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