HDFS-12216. Ozone: TestKeys is failing consistently. Contributed by Mukul Kumar Singh.
This commit is contained in:
parent
a245c60bb0
commit
7d132596da
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -530,6 +530,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;
|
||||
}
|
||||
|
@ -2312,7 +2320,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);
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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());
|
||||
|
|
Loading…
Reference in New Issue