HDDS-1076. TestSCMNodeManager crashed the jvm. Contributed by Lokesh Jain.

This commit is contained in:
Shashikant Banerjee 2019-02-15 21:13:30 +05:30
parent e0fe3d1eca
commit de934ba2dc
8 changed files with 163 additions and 38 deletions

View File

@ -0,0 +1,101 @@
/*
* 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.utils;
import org.apache.ratis.util.function.CheckedRunnable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
/**
* This class encapsulates ScheduledExecutorService.
*/
public class Scheduler {
private static final Logger LOG =
LoggerFactory.getLogger(Scheduler.class);
private ScheduledExecutorService scheduler;
private volatile boolean isClosed;
private String threadName;
/**
* Creates a ScheduledExecutorService based on input arguments.
* @param threadName - thread name
* @param isDaemon - if true the threads in the scheduler are started as
* daemon
* @param numCoreThreads - number of core threads to maintain in the scheduler
*/
public Scheduler(String threadName, boolean isDaemon, int numCoreThreads) {
scheduler = Executors.newScheduledThreadPool(numCoreThreads, r -> {
Thread t = new Thread(r);
t.setName(threadName);
t.setDaemon(isDaemon);
return t;
});
this.threadName = threadName;
isClosed = false;
}
public void schedule(Runnable runnable, long delay, TimeUnit timeUnit) {
scheduler.schedule(runnable, delay, timeUnit);
}
public void schedule(CheckedRunnable runnable, long delay,
TimeUnit timeUnit, Logger logger, String errMsg) {
scheduler.schedule(() -> {
try {
runnable.run();
} catch (Throwable throwable) {
logger.error(errMsg, throwable);
}
}, delay, timeUnit);
}
public void scheduleWithFixedDelay(Runnable runnable, long initialDelay,
long fixedDelay, TimeUnit timeUnit) {
scheduler
.scheduleWithFixedDelay(runnable, initialDelay, fixedDelay, timeUnit);
}
public boolean isClosed() {
return isClosed;
}
/**
* Closes the scheduler for further task submission. Any pending tasks not
* yet executed are also cancelled. For the executing tasks the scheduler
* waits 60 seconds for completion.
*/
public void close() {
isClosed = true;
scheduler.shutdownNow();
try {
scheduler.awaitTermination(60, TimeUnit.SECONDS);
} catch (InterruptedException e) {
LOG.info(threadName + " interrupted while waiting for task completion {}",
e);
}
scheduler = null;
}
}

View File

@ -54,4 +54,10 @@ public Pipeline create(ReplicationType type, ReplicationFactor factor,
List<DatanodeDetails> nodes) { List<DatanodeDetails> nodes) {
return providers.get(type).create(factor, nodes); return providers.get(type).create(factor, nodes);
} }
public void close() {
for (PipelineProvider p : providers.values()) {
p.close();
}
}
} }

View File

@ -32,4 +32,6 @@ public interface PipelineProvider {
Pipeline create(ReplicationFactor factor) throws IOException; Pipeline create(ReplicationFactor factor) throws IOException;
Pipeline create(ReplicationFactor factor, List<DatanodeDetails> nodes); Pipeline create(ReplicationFactor factor, List<DatanodeDetails> nodes);
void close();
} }

View File

@ -28,6 +28,7 @@
import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementRandom; import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementRandom;
import org.apache.hadoop.hdds.scm.node.NodeManager; import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline.PipelineState; import org.apache.hadoop.hdds.scm.pipeline.Pipeline.PipelineState;
import org.apache.hadoop.utils.Scheduler;
import java.io.IOException; import java.io.IOException;
import java.lang.reflect.Constructor; import java.lang.reflect.Constructor;
@ -45,12 +46,18 @@ public class RatisPipelineProvider implements PipelineProvider {
private final NodeManager nodeManager; private final NodeManager nodeManager;
private final PipelineStateManager stateManager; private final PipelineStateManager stateManager;
private final Configuration conf; private final Configuration conf;
private static Scheduler scheduler;
RatisPipelineProvider(NodeManager nodeManager, RatisPipelineProvider(NodeManager nodeManager,
PipelineStateManager stateManager, Configuration conf) { PipelineStateManager stateManager, Configuration conf) {
this.nodeManager = nodeManager; this.nodeManager = nodeManager;
this.stateManager = stateManager; this.stateManager = stateManager;
this.conf = conf; this.conf = conf;
scheduler = new Scheduler("RatisPipelineUtilsThread", false, 1);
}
static Scheduler getScheduler() {
return scheduler;
} }
/** /**
@ -135,4 +142,9 @@ public Pipeline create(ReplicationFactor factor,
private void initializePipeline(Pipeline pipeline) throws IOException { private void initializePipeline(Pipeline pipeline) throws IOException {
RatisPipelineUtils.createPipeline(pipeline, conf); RatisPipelineUtils.createPipeline(pipeline, conf);
} }
@Override
public void close() {
scheduler.close();
}
} }

View File

@ -34,8 +34,6 @@
import org.apache.ratis.retry.RetryPolicy; import org.apache.ratis.retry.RetryPolicy;
import org.apache.ratis.rpc.SupportedRpcType; import org.apache.ratis.rpc.SupportedRpcType;
import org.apache.ratis.util.function.CheckedBiConsumer; import org.apache.ratis.util.function.CheckedBiConsumer;
import org.apache.ratis.util.TimeDuration;
import org.apache.ratis.util.TimeoutScheduler;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -52,8 +50,6 @@
*/ */
public final class RatisPipelineUtils { public final class RatisPipelineUtils {
private static TimeoutScheduler timeoutScheduler =
TimeoutScheduler.newInstance(1);
private static AtomicBoolean isPipelineCreatorRunning = private static AtomicBoolean isPipelineCreatorRunning =
new AtomicBoolean(false); new AtomicBoolean(false);
@ -127,12 +123,11 @@ public static void finalizeAndDestroyPipeline(PipelineManager pipelineManager,
.getTimeDuration(ScmConfigKeys.OZONE_SCM_PIPELINE_DESTROY_TIMEOUT, .getTimeDuration(ScmConfigKeys.OZONE_SCM_PIPELINE_DESTROY_TIMEOUT,
ScmConfigKeys.OZONE_SCM_PIPELINE_DESTROY_TIMEOUT_DEFAULT, ScmConfigKeys.OZONE_SCM_PIPELINE_DESTROY_TIMEOUT_DEFAULT,
TimeUnit.MILLISECONDS); TimeUnit.MILLISECONDS);
TimeDuration timeoutDuration = TimeDuration RatisPipelineProvider.getScheduler()
.valueOf(pipelineDestroyTimeoutInMillis, TimeUnit.MILLISECONDS); .schedule(() -> destroyPipeline(pipelineManager, pipeline, ozoneConf),
timeoutScheduler.onTimeout(timeoutDuration, pipelineDestroyTimeoutInMillis, TimeUnit.MILLISECONDS, LOG, String
() -> destroyPipeline(pipelineManager, pipeline, ozoneConf), LOG, .format("Destroy pipeline failed for pipeline:%s with %s",
() -> String.format("Destroy pipeline failed for pipeline:%s with %s", pipeline.getId(), group));
pipeline.getId(), group));
} else { } else {
destroyPipeline(pipelineManager, pipeline, ozoneConf); destroyPipeline(pipelineManager, pipeline, ozoneConf);
} }
@ -213,22 +208,12 @@ public static void scheduleFixedIntervalPipelineCreator(
ScmConfigKeys.OZONE_SCM_PIPELINE_CREATION_INTERVAL_DEFAULT, ScmConfigKeys.OZONE_SCM_PIPELINE_CREATION_INTERVAL_DEFAULT,
TimeUnit.MILLISECONDS); TimeUnit.MILLISECONDS);
// TODO: #CLUTIL We can start the job asap // TODO: #CLUTIL We can start the job asap
TimeDuration timeDuration = RatisPipelineProvider.getScheduler().scheduleWithFixedDelay(() -> {
TimeDuration.valueOf(intervalInMillis, TimeUnit.MILLISECONDS); if (!isPipelineCreatorRunning.compareAndSet(false, true)) {
timeoutScheduler.onTimeout(timeDuration, return;
() -> fixedIntervalPipelineCreator(pipelineManager, conf, }
timeDuration), LOG, createPipelines(pipelineManager, conf);
() -> "FixedIntervalPipelineCreatorJob failed."); }, intervalInMillis, intervalInMillis, TimeUnit.MILLISECONDS);
}
private static void fixedIntervalPipelineCreator(
PipelineManager pipelineManager, Configuration conf,
TimeDuration timeDuration) {
timeoutScheduler.onTimeout(timeDuration,
() -> fixedIntervalPipelineCreator(pipelineManager, conf,
timeDuration), LOG,
() -> "FixedIntervalPipelineCreatorJob failed.");
triggerPipelineCreation(pipelineManager, conf, 0);
} }
/** /**
@ -246,10 +231,9 @@ public static void triggerPipelineCreation(PipelineManager pipelineManager,
if (!isPipelineCreatorRunning.compareAndSet(false, true)) { if (!isPipelineCreatorRunning.compareAndSet(false, true)) {
return; return;
} }
timeoutScheduler RatisPipelineProvider.getScheduler()
.onTimeout(TimeDuration.valueOf(afterMillis, TimeUnit.MILLISECONDS), .schedule(() -> createPipelines(pipelineManager, conf), afterMillis,
() -> createPipelines(pipelineManager, conf), LOG, TimeUnit.MILLISECONDS);
() -> "PipelineCreation failed.");
} }
private static void createPipelines(PipelineManager pipelineManager, private static void createPipelines(PipelineManager pipelineManager,
@ -261,13 +245,18 @@ private static void createPipelines(PipelineManager pipelineManager,
for (HddsProtos.ReplicationFactor factor : HddsProtos.ReplicationFactor for (HddsProtos.ReplicationFactor factor : HddsProtos.ReplicationFactor
.values()) { .values()) {
try { while (true) {
pipelineManager.createPipeline(type, factor); try {
} catch (IOException ioe) { if (RatisPipelineProvider.getScheduler().isClosed()) {
break; break;
} catch (Throwable t) { }
LOG.error("Error while creating pipelines {}", t); pipelineManager.createPipeline(type, factor);
break; } catch (IOException ioe) {
break;
} catch (Throwable t) {
LOG.error("Error while creating pipelines {}", t);
break;
}
} }
} }
isPipelineCreatorRunning.set(false); isPipelineCreatorRunning.set(false);

View File

@ -267,6 +267,10 @@ public void removePipeline(PipelineID pipelineID) throws IOException {
@Override @Override
public void close() throws IOException { public void close() throws IOException {
if (pipelineFactory != null) {
pipelineFactory.close();
}
if (pipelineStore != null) { if (pipelineStore != null) {
pipelineStore.close(); pipelineStore.close();
} }

View File

@ -72,4 +72,9 @@ public Pipeline create(ReplicationFactor factor,
.setNodes(nodes) .setNodes(nodes)
.build(); .build();
} }
@Override
public void close() {
// Nothing to do in here.
}
} }

View File

@ -23,6 +23,7 @@
import org.apache.hadoop.ozone.HddsDatanodeService; import org.apache.hadoop.ozone.HddsDatanodeService;
import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.junit.After;
import org.junit.Test; import org.junit.Test;
import java.util.List; import java.util.List;
@ -51,6 +52,11 @@ public void init(int numDatanodes) throws Exception {
pipelineManager = scm.getPipelineManager(); pipelineManager = scm.getPipelineManager();
} }
@After
public void cleanup() {
cluster.shutdown();
}
@Test(timeout = 30000) @Test(timeout = 30000)
public void testAutomaticPipelineCreationOnPipelineDestroy() public void testAutomaticPipelineCreationOnPipelineDestroy()
throws Exception { throws Exception {
@ -90,6 +96,6 @@ private void waitForPipelines(int numPipelines)
GenericTestUtils.waitFor(() -> pipelineManager GenericTestUtils.waitFor(() -> pipelineManager
.getPipelines(HddsProtos.ReplicationType.RATIS, .getPipelines(HddsProtos.ReplicationType.RATIS,
HddsProtos.ReplicationFactor.THREE, Pipeline.PipelineState.OPEN) HddsProtos.ReplicationFactor.THREE, Pipeline.PipelineState.OPEN)
.size() == numPipelines, 100, 10000); .size() == numPipelines, 100, 20000);
} }
} }