Bladeren bron

HDFS-12216. Ozone: TestKeys is failing consistently. Contributed by Mukul Kumar Singh.

Anu Engineer 8 jaren geleden
bovenliggende
commit
38bc1d1097

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java

@@ -126,7 +126,8 @@ public final class ContainerUtils {
   public static ContainerProtos.ContainerCommandResponseProto logAndReturnError(
       Logger log, StorageContainerException ex,
       ContainerProtos.ContainerCommandRequestProto msg) {
-    log.info("Trace ID: {} : Message: {} : Result: {}", msg.getTraceID(),
+    log.info("Operation: {} : Trace ID: {} : Message: {} : Result: {}",
+        msg.getCmdType().name(), msg.getTraceID(),
         ex.getMessage(), ex.getResult().getValueDescriptor().getName());
     return getContainerResponse(msg, ex.getResult(), ex.getMessage()).build();
   }
@@ -142,7 +143,8 @@ public final class ContainerUtils {
   public static ContainerProtos.ContainerCommandResponseProto logAndReturnError(
       Logger log, RuntimeException ex,
       ContainerProtos.ContainerCommandRequestProto msg) {
-    log.info("Trace ID: {} : Message: {} ", msg.getTraceID(), ex.getMessage());
+    log.info("Operation: {} : Trace ID: {} : Message: {} ",
+        msg.getCmdType().name(), msg.getTraceID(), ex.getMessage());
     return getContainerResponse(msg, INVALID_ARGUMENT, ex.getMessage()).build();
   }
 

+ 1 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java

@@ -21,12 +21,9 @@ import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
 import org.apache.hadoop.scm.container.common.helpers.StorageContainerException;
-import org.apache.hadoop.ozone.container.common.impl.KeyManagerImpl;
 import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
 import org.apache.hadoop.utils.MetadataStore;
 import org.apache.hadoop.utils.MetadataStoreBuilder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.IOException;
@@ -100,21 +97,10 @@ public final class KeyUtils {
    * Shutdown all DB Handles.
    *
    * @param cache - Cache for DB Handles.
-   * @throws IOException
    */
   @SuppressWarnings("unchecked")
   public static void shutdownCache(ContainerCache cache)  {
-    Logger log = LoggerFactory.getLogger(KeyManagerImpl.class);
-    MetadataStore[] handles = new MetadataStore[cache.values().size()];
-    cache.values().toArray(handles);
-    Preconditions.checkState(handles.length == cache.values().size());
-    for (MetadataStore db : handles) {
-      try {
-        db.close();
-      } catch (IOException ex) {
-        log.error("error closing db. error {}", ex);
-      }
-    }
+    cache.shutdownCache();
   }
 
   /**

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServer.java

@@ -34,7 +34,9 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.net.ServerSocket;
+import java.net.SocketAddress;
 
 /**
  * Creates a netty server endpoint that acts as the communication layer for
@@ -65,8 +67,10 @@ public final class XceiverServer implements XceiverServerSpi {
     // use that as the container port
     if (conf.getBoolean(OzoneConfigKeys.DFS_CONTAINER_IPC_RANDOM_PORT,
         OzoneConfigKeys.DFS_CONTAINER_IPC_RANDOM_PORT_DEFAULT)) {
-      try (ServerSocket socket = new ServerSocket(0)) {
+      try (ServerSocket socket = new ServerSocket()) {
         socket.setReuseAddress(true);
+        SocketAddress address = new InetSocketAddress(0);
+        socket.bind(address);
         this.port = socket.getLocalPort();
         LOG.info("Found a free port for the server : {}", this.port);
       } catch (IOException e) {

+ 39 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.ozone.container.common.utils;
 
 import com.google.common.base.Preconditions;
+import org.apache.commons.collections.MapIterator;
 import org.apache.commons.collections.map.LRUMap;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -62,6 +63,42 @@ public final class ContainerCache extends LRUMap {
     return cache;
   }
 
+  /**
+   * Closes a db instance.
+   *
+   * @param container - name of the container to be closed.
+   * @param db - db instance to close.
+   */
+  private void closeDB(String container, MetadataStore db) {
+    if (db != null) {
+      try {
+        db.close();
+      } catch (IOException e) {
+        LOG.error("Error closing DB. Container: " + container, e);
+      }
+    }
+  }
+
+  /**
+   * Closes all the db instances and resets the cache.
+   */
+  public void shutdownCache() {
+    lock.lock();
+    try {
+      // iterate the cache and close each db
+      MapIterator iterator = cache.mapIterator();
+      while (iterator.hasNext()) {
+        iterator.next();
+        MetadataStore db = (MetadataStore) iterator.getValue();
+        closeDB(iterator.getKey().toString(), db);
+      }
+      // reset the cache
+      cache.clear();
+    } finally {
+      lock.unlock();
+    }
+  }
+
   /**
    * {@inheritDoc}
    */
@@ -70,9 +107,7 @@ public final class ContainerCache extends LRUMap {
     lock.lock();
     try {
       MetadataStore db = (MetadataStore) entry.getValue();
-      db.close();
-    } catch (IOException e) {
-      LOG.error("Error closing DB. Container: " + entry.getKey().toString(), e);
+      closeDB(entry.getKey().toString(), db);
     } finally {
       lock.unlock();
     }
@@ -107,13 +142,7 @@ public final class ContainerCache extends LRUMap {
     lock.lock();
     try {
       MetadataStore db = this.getDB(containerName);
-      if (db != null) {
-        try {
-          db.close();
-        } catch (IOException e) {
-          LOG.warn("There is some issue to stop an unused DB handler.", e);
-        }
-      }
+      closeDB(containerName, db);
       this.remove(containerName);
     } finally {
       lock.unlock();

+ 9 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -556,6 +556,14 @@ public class MiniDFSCluster implements AutoCloseable {
       this.ipcPort = ipcPort;
     }
 
+    public Configuration getConf() {
+      return conf;
+    }
+
+    public DataNode getDatanode() {
+      return datanode;
+    }
+
     public void setDnArgs(String ... args) {
       dnArgs = args;
     }
@@ -2370,7 +2378,7 @@ public class MiniDFSCluster implements AutoCloseable {
       conf.set(DFS_DATANODE_ADDRESS_KEY, 
           addr.getAddress().getHostAddress() + ":" + addr.getPort());
       conf.set(DFS_DATANODE_IPC_ADDRESS_KEY,
-          addr.getAddress().getHostAddress() + ":" + dnprop.ipcPort); 
+          addr.getAddress().getHostAddress() + ":" + dnprop.ipcPort);
     }
     final DataNode newDn = DataNode.createDataNode(args, conf, secureResources);
 

+ 24 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java

@@ -53,6 +53,10 @@ import java.util.Random;
 import java.util.UUID;
 import java.util.concurrent.TimeoutException;
 
+import static org.apache.hadoop.ozone.OzoneConfigKeys
+    .DFS_CONTAINER_IPC_PORT;
+import static org.apache.hadoop.ozone.OzoneConfigKeys
+    .DFS_CONTAINER_IPC_RANDOM_PORT;
 import static org.apache.hadoop.ozone.protocol.proto.OzoneProtos.NodeState
     .HEALTHY;
 import static org.junit.Assert.assertFalse;
@@ -135,6 +139,26 @@ public final class MiniOzoneCluster extends MiniDFSCluster
     }
   }
 
+  public boolean restartDataNode(int i) throws IOException {
+    return restartDataNode(i, true);
+  }
+  /*
+   * Restart a particular datanode, wait for it to become active
+   */
+  public boolean restartDataNode(int i, boolean keepPort) throws IOException {
+    if (keepPort) {
+      DataNodeProperties dnProp = dataNodes.get(i);
+      int currentPort = dnProp.getDatanode().getOzoneContainerManager()
+          .getContainerServerPort();
+      Configuration config = dnProp.getConf();
+      config.setInt(DFS_CONTAINER_IPC_PORT, currentPort);
+      config.setBoolean(DFS_CONTAINER_IPC_RANDOM_PORT, false);
+    }
+    boolean status =  super.restartDataNode(i, true);
+    this.waitActive();
+    return status;
+  }
+
   @Override
   public void shutdown() {
     super.shutdown();

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java

@@ -65,8 +65,9 @@ public class TestKeys {
   public Timeout testTimeout = new Timeout(300000);
 
   private static MiniOzoneCluster ozoneCluster = null;
-  static private String path;
+  private static String path;
   private static OzoneRestClient ozoneRestClient = null;
+  private static long currentTime;
 
   /**
    * Create a MiniDFSCluster for testing.
@@ -93,6 +94,7 @@ public class TestKeys {
     final int port = dataNode.getInfoPort();
     ozoneRestClient = new OzoneRestClient(
         String.format("http://localhost:%d", port));
+    currentTime = Time.now();
   }
 
   /**
@@ -405,7 +407,6 @@ public class TestKeys {
     assertNotNull(helper.getBucket());
     assertNotNull(helper.getFile());
 
-    long currentTime = Time.now();
     // add keys [list-key0, list-key1, ..., list-key9]
     for (int x = 0; x < 10; x++) {
       String newkeyName = "list-key" + x;
@@ -490,7 +491,6 @@ public class TestKeys {
 
   static void runTestGetKeyInfo(PutHelper helper)
       throws OzoneException, ParseException {
-    long currentTime = Time.now();
     String keyName = helper.putKey();
     assertNotNull(helper.getBucket());
     assertNotNull(helper.getFile());