Merging trunk to HDFS-2802 branch.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1416603 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
commit
d500d59cbe
|
@ -306,6 +306,10 @@ Release 2.0.3-alpha - Unreleased
|
||||||
|
|
||||||
HADOOP-9020. Add a SASL PLAIN server (daryn via bobby)
|
HADOOP-9020. Add a SASL PLAIN server (daryn via bobby)
|
||||||
|
|
||||||
|
HADOOP-9090. Support on-demand publish of metrics. (Mostafa Elhemali via
|
||||||
|
suresh)
|
||||||
|
|
||||||
|
|
||||||
IMPROVEMENTS
|
IMPROVEMENTS
|
||||||
|
|
||||||
HADOOP-8789. Tests setLevel(Level.OFF) should be Level.ERROR.
|
HADOOP-8789. Tests setLevel(Level.OFF) should be Level.ERROR.
|
||||||
|
|
|
@ -90,6 +90,17 @@ public abstract class MetricsSystem implements MetricsSystemMXBean {
|
||||||
*/
|
*/
|
||||||
public abstract void register(Callback callback);
|
public abstract void register(Callback callback);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Requests an immediate publish of all metrics from sources to sinks.
|
||||||
|
*
|
||||||
|
* This is a "soft" request: the expectation is that a best effort will be
|
||||||
|
* done to synchronously snapshot the metrics from all the sources and put
|
||||||
|
* them in all the sinks (including flushing the sinks) before returning to
|
||||||
|
* the caller. If this can't be accomplished in reasonable time it's OK to
|
||||||
|
* return to the caller before everything is done.
|
||||||
|
*/
|
||||||
|
public abstract void publishMetricsNow();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Shutdown the metrics system completely (usually during server shutdown.)
|
* Shutdown the metrics system completely (usually during server shutdown.)
|
||||||
* The MetricsSystemMXBean will be unregistered.
|
* The MetricsSystemMXBean will be unregistered.
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
package org.apache.hadoop.metrics2.impl;
|
package org.apache.hadoop.metrics2.impl;
|
||||||
|
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
|
import java.util.concurrent.*;
|
||||||
|
|
||||||
import static com.google.common.base.Preconditions.*;
|
import static com.google.common.base.Preconditions.*;
|
||||||
|
|
||||||
|
@ -48,6 +49,7 @@ class MetricsSinkAdapter implements SinkQueue.Consumer<MetricsBuffer> {
|
||||||
private volatile boolean stopping = false;
|
private volatile boolean stopping = false;
|
||||||
private volatile boolean inError = false;
|
private volatile boolean inError = false;
|
||||||
private final int period, firstRetryDelay, retryCount;
|
private final int period, firstRetryDelay, retryCount;
|
||||||
|
private final long oobPutTimeout;
|
||||||
private final float retryBackoff;
|
private final float retryBackoff;
|
||||||
private final MetricsRegistry registry = new MetricsRegistry("sinkadapter");
|
private final MetricsRegistry registry = new MetricsRegistry("sinkadapter");
|
||||||
private final MutableStat latency;
|
private final MutableStat latency;
|
||||||
|
@ -69,6 +71,8 @@ class MetricsSinkAdapter implements SinkQueue.Consumer<MetricsBuffer> {
|
||||||
this.period = checkArg(period, period > 0, "period");
|
this.period = checkArg(period, period > 0, "period");
|
||||||
firstRetryDelay = checkArg(retryDelay, retryDelay > 0, "retry delay");
|
firstRetryDelay = checkArg(retryDelay, retryDelay > 0, "retry delay");
|
||||||
this.retryBackoff = checkArg(retryBackoff, retryBackoff>1, "retry backoff");
|
this.retryBackoff = checkArg(retryBackoff, retryBackoff>1, "retry backoff");
|
||||||
|
oobPutTimeout = (long)
|
||||||
|
(firstRetryDelay * Math.pow(retryBackoff, retryCount) * 1000);
|
||||||
this.retryCount = retryCount;
|
this.retryCount = retryCount;
|
||||||
this.queue = new SinkQueue<MetricsBuffer>(checkArg(queueCapacity,
|
this.queue = new SinkQueue<MetricsBuffer>(checkArg(queueCapacity,
|
||||||
queueCapacity > 0, "queue capacity"));
|
queueCapacity > 0, "queue capacity"));
|
||||||
|
@ -95,6 +99,23 @@ class MetricsSinkAdapter implements SinkQueue.Consumer<MetricsBuffer> {
|
||||||
}
|
}
|
||||||
return true; // OK
|
return true; // OK
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public boolean putMetricsImmediate(MetricsBuffer buffer) {
|
||||||
|
WaitableMetricsBuffer waitableBuffer =
|
||||||
|
new WaitableMetricsBuffer(buffer);
|
||||||
|
if (!queue.enqueue(waitableBuffer)) {
|
||||||
|
LOG.warn(name + " has a full queue and can't consume the given metrics.");
|
||||||
|
dropped.incr();
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (!waitableBuffer.waitTillNotified(oobPutTimeout)) {
|
||||||
|
LOG.warn(name +
|
||||||
|
" couldn't fulfill an immediate putMetrics request in time." +
|
||||||
|
" Abandoning.");
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
void publishMetricsFromQueue() {
|
void publishMetricsFromQueue() {
|
||||||
int retryDelay = firstRetryDelay;
|
int retryDelay = firstRetryDelay;
|
||||||
|
@ -158,6 +179,9 @@ class MetricsSinkAdapter implements SinkQueue.Consumer<MetricsBuffer> {
|
||||||
sink.flush();
|
sink.flush();
|
||||||
latency.add(Time.now() - ts);
|
latency.add(Time.now() - ts);
|
||||||
}
|
}
|
||||||
|
if (buffer instanceof WaitableMetricsBuffer) {
|
||||||
|
((WaitableMetricsBuffer)buffer).notifyAnyWaiters();
|
||||||
|
}
|
||||||
LOG.debug("Done");
|
LOG.debug("Done");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -191,4 +215,26 @@ class MetricsSinkAdapter implements SinkQueue.Consumer<MetricsBuffer> {
|
||||||
MetricsSink sink() {
|
MetricsSink sink() {
|
||||||
return sink;
|
return sink;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
static class WaitableMetricsBuffer extends MetricsBuffer {
|
||||||
|
private final Semaphore notificationSemaphore =
|
||||||
|
new Semaphore(0);
|
||||||
|
|
||||||
|
public WaitableMetricsBuffer(MetricsBuffer metricsBuffer) {
|
||||||
|
super(metricsBuffer);
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean waitTillNotified(long millisecondsToWait) {
|
||||||
|
try {
|
||||||
|
return notificationSemaphore.tryAcquire(millisecondsToWait,
|
||||||
|
TimeUnit.MILLISECONDS);
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void notifyAnyWaiters() {
|
||||||
|
notificationSemaphore.release();
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -344,9 +344,19 @@ public class MetricsSystemImpl extends MetricsSystem implements MetricsSource {
|
||||||
synchronized void onTimerEvent() {
|
synchronized void onTimerEvent() {
|
||||||
logicalTime += period;
|
logicalTime += period;
|
||||||
if (sinks.size() > 0) {
|
if (sinks.size() > 0) {
|
||||||
publishMetrics(sampleMetrics());
|
publishMetrics(sampleMetrics(), false);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Requests an immediate publish of all metrics from sources to sinks.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void publishMetricsNow() {
|
||||||
|
if (sinks.size() > 0) {
|
||||||
|
publishMetrics(sampleMetrics(), true);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Sample all the sources for a snapshot of metrics/tags
|
* Sample all the sources for a snapshot of metrics/tags
|
||||||
|
@ -380,12 +390,20 @@ public class MetricsSystemImpl extends MetricsSystem implements MetricsSource {
|
||||||
/**
|
/**
|
||||||
* Publish a metrics snapshot to all the sinks
|
* Publish a metrics snapshot to all the sinks
|
||||||
* @param buffer the metrics snapshot to publish
|
* @param buffer the metrics snapshot to publish
|
||||||
|
* @param immediate indicates that we should publish metrics immediately
|
||||||
|
* instead of using a separate thread.
|
||||||
*/
|
*/
|
||||||
synchronized void publishMetrics(MetricsBuffer buffer) {
|
synchronized void publishMetrics(MetricsBuffer buffer, boolean immediate) {
|
||||||
int dropped = 0;
|
int dropped = 0;
|
||||||
for (MetricsSinkAdapter sa : sinks.values()) {
|
for (MetricsSinkAdapter sa : sinks.values()) {
|
||||||
long startTime = Time.now();
|
long startTime = Time.now();
|
||||||
dropped += sa.putMetrics(buffer, logicalTime) ? 0 : 1;
|
boolean result;
|
||||||
|
if (immediate) {
|
||||||
|
result = sa.putMetricsImmediate(buffer);
|
||||||
|
} else {
|
||||||
|
result = sa.putMetrics(buffer, logicalTime);
|
||||||
|
}
|
||||||
|
dropped += result ? 0 : 1;
|
||||||
publishStat.add(Time.now() - startTime);
|
publishStat.add(Time.now() - startTime);
|
||||||
}
|
}
|
||||||
droppedPubAll.incr(dropped);
|
droppedPubAll.incr(dropped);
|
||||||
|
|
|
@ -29,8 +29,6 @@ import java.util.ArrayList;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.CountDownLatch;
|
|
||||||
import java.util.concurrent.TimeUnit;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
@ -115,31 +113,23 @@ public class TestGangliaMetrics {
|
||||||
final int expectedCountFromGanglia30 = expectedMetrics.length;
|
final int expectedCountFromGanglia30 = expectedMetrics.length;
|
||||||
final int expectedCountFromGanglia31 = 2 * expectedMetrics.length;
|
final int expectedCountFromGanglia31 = 2 * expectedMetrics.length;
|
||||||
|
|
||||||
// use latch to make sure we received required records before shutting
|
|
||||||
// down the MetricSystem
|
|
||||||
CountDownLatch latch = new CountDownLatch(
|
|
||||||
expectedCountFromGanglia30 + expectedCountFromGanglia31);
|
|
||||||
|
|
||||||
// Setup test for GangliaSink30
|
// Setup test for GangliaSink30
|
||||||
AbstractGangliaSink gsink30 = new GangliaSink30();
|
AbstractGangliaSink gsink30 = new GangliaSink30();
|
||||||
gsink30.init(cb.subset("test"));
|
gsink30.init(cb.subset("test"));
|
||||||
MockDatagramSocket mockds30 = new MockDatagramSocket(latch);
|
MockDatagramSocket mockds30 = new MockDatagramSocket();
|
||||||
GangliaMetricsTestHelper.setDatagramSocket(gsink30, mockds30);
|
GangliaMetricsTestHelper.setDatagramSocket(gsink30, mockds30);
|
||||||
|
|
||||||
// Setup test for GangliaSink31
|
// Setup test for GangliaSink31
|
||||||
AbstractGangliaSink gsink31 = new GangliaSink31();
|
AbstractGangliaSink gsink31 = new GangliaSink31();
|
||||||
gsink31.init(cb.subset("test"));
|
gsink31.init(cb.subset("test"));
|
||||||
MockDatagramSocket mockds31 = new MockDatagramSocket(latch);
|
MockDatagramSocket mockds31 = new MockDatagramSocket();
|
||||||
GangliaMetricsTestHelper.setDatagramSocket(gsink31, mockds31);
|
GangliaMetricsTestHelper.setDatagramSocket(gsink31, mockds31);
|
||||||
|
|
||||||
// register the sinks
|
// register the sinks
|
||||||
ms.register("gsink30", "gsink30 desc", gsink30);
|
ms.register("gsink30", "gsink30 desc", gsink30);
|
||||||
ms.register("gsink31", "gsink31 desc", gsink31);
|
ms.register("gsink31", "gsink31 desc", gsink31);
|
||||||
ms.onTimerEvent(); // trigger something interesting
|
ms.publishMetricsNow(); // publish the metrics
|
||||||
|
|
||||||
// wait for all records and the stop MetricSystem. Without this
|
|
||||||
// sometime the ms gets shutdown before all the sinks have consumed
|
|
||||||
latch.await(200, TimeUnit.MILLISECONDS);
|
|
||||||
ms.stop();
|
ms.stop();
|
||||||
|
|
||||||
// check GanfliaSink30 data
|
// check GanfliaSink30 data
|
||||||
|
@ -198,7 +188,6 @@ public class TestGangliaMetrics {
|
||||||
*/
|
*/
|
||||||
private class MockDatagramSocket extends DatagramSocket {
|
private class MockDatagramSocket extends DatagramSocket {
|
||||||
private ArrayList<byte[]> capture;
|
private ArrayList<byte[]> capture;
|
||||||
private CountDownLatch latch;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @throws SocketException
|
* @throws SocketException
|
||||||
|
@ -207,15 +196,6 @@ public class TestGangliaMetrics {
|
||||||
capture = new ArrayList<byte[]>();
|
capture = new ArrayList<byte[]>();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* @param latch
|
|
||||||
* @throws SocketException
|
|
||||||
*/
|
|
||||||
public MockDatagramSocket(CountDownLatch latch) throws SocketException {
|
|
||||||
this();
|
|
||||||
this.latch = latch;
|
|
||||||
}
|
|
||||||
|
|
||||||
/* (non-Javadoc)
|
/* (non-Javadoc)
|
||||||
* @see java.net.DatagramSocket#send(java.net.DatagramPacket)
|
* @see java.net.DatagramSocket#send(java.net.DatagramPacket)
|
||||||
*/
|
*/
|
||||||
|
@ -225,9 +205,6 @@ public class TestGangliaMetrics {
|
||||||
byte[] bytes = new byte[p.getLength()];
|
byte[] bytes = new byte[p.getLength()];
|
||||||
System.arraycopy(p.getData(), p.getOffset(), bytes, 0, p.getLength());
|
System.arraycopy(p.getData(), p.getOffset(), bytes, 0, p.getLength());
|
||||||
capture.add(bytes);
|
capture.add(bytes);
|
||||||
|
|
||||||
// decrement the latch
|
|
||||||
latch.countDown();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -18,7 +18,11 @@
|
||||||
|
|
||||||
package org.apache.hadoop.metrics2.impl;
|
package org.apache.hadoop.metrics2.impl;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.*;
|
||||||
|
import java.util.concurrent.*;
|
||||||
|
import java.util.concurrent.atomic.*;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.runner.RunWith;
|
import org.junit.runner.RunWith;
|
||||||
|
@ -26,9 +30,11 @@ import org.junit.runner.RunWith;
|
||||||
import org.mockito.ArgumentCaptor;
|
import org.mockito.ArgumentCaptor;
|
||||||
import org.mockito.Captor;
|
import org.mockito.Captor;
|
||||||
import org.mockito.runners.MockitoJUnitRunner;
|
import org.mockito.runners.MockitoJUnitRunner;
|
||||||
|
|
||||||
import static org.junit.Assert.*;
|
import static org.junit.Assert.*;
|
||||||
import static org.mockito.Mockito.*;
|
import static org.mockito.Mockito.*;
|
||||||
|
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
|
|
||||||
import org.apache.commons.configuration.SubsetConfiguration;
|
import org.apache.commons.configuration.SubsetConfiguration;
|
||||||
|
@ -36,6 +42,8 @@ import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.metrics2.MetricsException;
|
import org.apache.hadoop.metrics2.MetricsException;
|
||||||
import static org.apache.hadoop.test.MoreAsserts.*;
|
import static org.apache.hadoop.test.MoreAsserts.*;
|
||||||
|
|
||||||
|
import org.apache.hadoop.metrics2.AbstractMetric;
|
||||||
import org.apache.hadoop.metrics2.MetricsRecord;
|
import org.apache.hadoop.metrics2.MetricsRecord;
|
||||||
import org.apache.hadoop.metrics2.MetricsSink;
|
import org.apache.hadoop.metrics2.MetricsSink;
|
||||||
import org.apache.hadoop.metrics2.MetricsSource;
|
import org.apache.hadoop.metrics2.MetricsSource;
|
||||||
|
@ -47,6 +55,7 @@ import org.apache.hadoop.metrics2.lib.MetricsRegistry;
|
||||||
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
|
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
|
||||||
import org.apache.hadoop.metrics2.lib.MutableRate;
|
import org.apache.hadoop.metrics2.lib.MutableRate;
|
||||||
import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
|
import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
|
||||||
|
import org.apache.hadoop.util.StringUtils;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test the MetricsSystemImpl class
|
* Test the MetricsSystemImpl class
|
||||||
|
@ -72,7 +81,7 @@ public class TestMetricsSystemImpl {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test public void testInitFirst() throws Exception {
|
@Test public void testInitFirst() throws Exception {
|
||||||
ConfigBuilder cb = new ConfigBuilder().add("*.period", 8)
|
new ConfigBuilder().add("*.period", 8)
|
||||||
//.add("test.sink.plugin.urls", getPluginUrlsAsString())
|
//.add("test.sink.plugin.urls", getPluginUrlsAsString())
|
||||||
.add("test.sink.test.class", TestSink.class.getName())
|
.add("test.sink.test.class", TestSink.class.getName())
|
||||||
.add("test.*.source.filter.exclude", "s0")
|
.add("test.*.source.filter.exclude", "s0")
|
||||||
|
@ -93,8 +102,9 @@ public class TestMetricsSystemImpl {
|
||||||
MetricsSink sink2 = mock(MetricsSink.class);
|
MetricsSink sink2 = mock(MetricsSink.class);
|
||||||
ms.registerSink("sink1", "sink1 desc", sink1);
|
ms.registerSink("sink1", "sink1 desc", sink1);
|
||||||
ms.registerSink("sink2", "sink2 desc", sink2);
|
ms.registerSink("sink2", "sink2 desc", sink2);
|
||||||
ms.onTimerEvent(); // trigger something interesting
|
ms.publishMetricsNow(); // publish the metrics
|
||||||
ms.stop();
|
ms.stop();
|
||||||
|
ms.shutdown();
|
||||||
|
|
||||||
verify(sink1, times(2)).putMetrics(r1.capture());
|
verify(sink1, times(2)).putMetrics(r1.capture());
|
||||||
List<MetricsRecord> mr1 = r1.getAllValues();
|
List<MetricsRecord> mr1 = r1.getAllValues();
|
||||||
|
@ -104,6 +114,177 @@ public class TestMetricsSystemImpl {
|
||||||
assertEquals("output", mr1, mr2);
|
assertEquals("output", mr1, mr2);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test public void testMultiThreadedPublish() throws Exception {
|
||||||
|
new ConfigBuilder().add("*.period", 80)
|
||||||
|
.add("test.sink.Collector.queue.capacity", "20")
|
||||||
|
.save(TestMetricsConfig.getTestFilename("hadoop-metrics2-test"));
|
||||||
|
final MetricsSystemImpl ms = new MetricsSystemImpl("Test");
|
||||||
|
ms.start();
|
||||||
|
final int numThreads = 10;
|
||||||
|
final CollectingSink sink = new CollectingSink(numThreads);
|
||||||
|
ms.registerSink("Collector",
|
||||||
|
"Collector of values from all threads.", sink);
|
||||||
|
final TestSource[] sources = new TestSource[numThreads];
|
||||||
|
final Thread[] threads = new Thread[numThreads];
|
||||||
|
final String[] results = new String[numThreads];
|
||||||
|
final CyclicBarrier barrier1 = new CyclicBarrier(numThreads),
|
||||||
|
barrier2 = new CyclicBarrier(numThreads);
|
||||||
|
for (int i = 0; i < numThreads; i++) {
|
||||||
|
sources[i] = ms.register("threadSource" + i,
|
||||||
|
"A source of my threaded goodness.",
|
||||||
|
new TestSource("threadSourceRec" + i));
|
||||||
|
threads[i] = new Thread(new Runnable() {
|
||||||
|
private boolean safeAwait(int mySource, CyclicBarrier barrier) {
|
||||||
|
try {
|
||||||
|
barrier1.await(2, TimeUnit.SECONDS);
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
results[mySource] = "Interrupted";
|
||||||
|
return false;
|
||||||
|
} catch (BrokenBarrierException e) {
|
||||||
|
results[mySource] = "Broken Barrier";
|
||||||
|
return false;
|
||||||
|
} catch (TimeoutException e) {
|
||||||
|
results[mySource] = "Timed out on barrier";
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
int mySource = Integer.parseInt(Thread.currentThread().getName());
|
||||||
|
if (sink.collected[mySource].get() != 0L) {
|
||||||
|
results[mySource] = "Someone else collected my metric!";
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
// Wait for all the threads to come here so we can hammer
|
||||||
|
// the system at the same time
|
||||||
|
if (!safeAwait(mySource, barrier1)) return;
|
||||||
|
sources[mySource].g1.set(230);
|
||||||
|
ms.publishMetricsNow();
|
||||||
|
// Since some other thread may have snatched my metric,
|
||||||
|
// I need to wait for the threads to finish before checking.
|
||||||
|
if (!safeAwait(mySource, barrier2)) return;
|
||||||
|
if (sink.collected[mySource].get() != 230L) {
|
||||||
|
results[mySource] = "Metric not collected!";
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
results[mySource] = "Passed";
|
||||||
|
}
|
||||||
|
}, "" + i);
|
||||||
|
}
|
||||||
|
for (Thread t : threads)
|
||||||
|
t.start();
|
||||||
|
for (Thread t : threads)
|
||||||
|
t.join();
|
||||||
|
assertEquals(0L, ms.droppedPubAll.value());
|
||||||
|
assertTrue(StringUtils.join("\n", Arrays.asList(results)),
|
||||||
|
Iterables.all(Arrays.asList(results), new Predicate<String>() {
|
||||||
|
@Override
|
||||||
|
public boolean apply(@Nullable String input) {
|
||||||
|
return input.equalsIgnoreCase("Passed");
|
||||||
|
}
|
||||||
|
}));
|
||||||
|
ms.stop();
|
||||||
|
ms.shutdown();
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class CollectingSink implements MetricsSink {
|
||||||
|
private final AtomicLong[] collected;
|
||||||
|
|
||||||
|
public CollectingSink(int capacity) {
|
||||||
|
collected = new AtomicLong[capacity];
|
||||||
|
for (int i = 0; i < capacity; i++) {
|
||||||
|
collected[i] = new AtomicLong();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void init(SubsetConfiguration conf) {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void putMetrics(MetricsRecord record) {
|
||||||
|
final String prefix = "threadSourceRec";
|
||||||
|
if (record.name().startsWith(prefix)) {
|
||||||
|
final int recordNumber = Integer.parseInt(
|
||||||
|
record.name().substring(prefix.length()));
|
||||||
|
ArrayList<String> names = new ArrayList<String>();
|
||||||
|
for (AbstractMetric m : record.metrics()) {
|
||||||
|
if (m.name().equalsIgnoreCase("g1")) {
|
||||||
|
collected[recordNumber].set(m.value().longValue());
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
names.add(m.name());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void flush() {
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test public void testHangingSink() {
|
||||||
|
new ConfigBuilder().add("*.period", 8)
|
||||||
|
.add("test.sink.test.class", TestSink.class.getName())
|
||||||
|
.add("test.sink.hanging.retry.delay", "1")
|
||||||
|
.add("test.sink.hanging.retry.backoff", "1.01")
|
||||||
|
.add("test.sink.hanging.retry.count", "0")
|
||||||
|
.save(TestMetricsConfig.getTestFilename("hadoop-metrics2-test"));
|
||||||
|
MetricsSystemImpl ms = new MetricsSystemImpl("Test");
|
||||||
|
ms.start();
|
||||||
|
TestSource s = ms.register("s3", "s3 desc", new TestSource("s3rec"));
|
||||||
|
s.c1.incr();
|
||||||
|
HangingSink hanging = new HangingSink();
|
||||||
|
ms.registerSink("hanging", "Hang the sink!", hanging);
|
||||||
|
ms.publishMetricsNow();
|
||||||
|
assertEquals(1L, ms.droppedPubAll.value());
|
||||||
|
assertFalse(hanging.getInterrupted());
|
||||||
|
ms.stop();
|
||||||
|
ms.shutdown();
|
||||||
|
assertTrue(hanging.getInterrupted());
|
||||||
|
assertTrue("The sink didn't get called after its first hang " +
|
||||||
|
"for subsequent records.", hanging.getGotCalledSecondTime());
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class HangingSink implements MetricsSink {
|
||||||
|
private volatile boolean interrupted;
|
||||||
|
private boolean gotCalledSecondTime;
|
||||||
|
private boolean firstTime = true;
|
||||||
|
|
||||||
|
public boolean getGotCalledSecondTime() {
|
||||||
|
return gotCalledSecondTime;
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean getInterrupted() {
|
||||||
|
return interrupted;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void init(SubsetConfiguration conf) {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void putMetrics(MetricsRecord record) {
|
||||||
|
// No need to hang every time, just the first record.
|
||||||
|
if (!firstTime) {
|
||||||
|
gotCalledSecondTime = true;
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
firstTime = false;
|
||||||
|
try {
|
||||||
|
Thread.sleep(10 * 1000);
|
||||||
|
} catch (InterruptedException ex) {
|
||||||
|
interrupted = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void flush() {
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Test public void testRegisterDups() {
|
@Test public void testRegisterDups() {
|
||||||
MetricsSystem ms = new MetricsSystemImpl();
|
MetricsSystem ms = new MetricsSystemImpl();
|
||||||
TestSource ts1 = new TestSource("ts1");
|
TestSource ts1 = new TestSource("ts1");
|
||||||
|
@ -116,6 +297,7 @@ public class TestMetricsSystemImpl {
|
||||||
MetricsSource s2 = ms.getSource("ts1");
|
MetricsSource s2 = ms.getSource("ts1");
|
||||||
assertNotNull(s2);
|
assertNotNull(s2);
|
||||||
assertNotSame(s1, s2);
|
assertNotSame(s1, s2);
|
||||||
|
ms.shutdown();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(expected=MetricsException.class) public void testRegisterDupError() {
|
@Test(expected=MetricsException.class) public void testRegisterDupError() {
|
||||||
|
|
|
@ -176,6 +176,9 @@ Trunk (Unreleased)
|
||||||
HDFS-4209. Clean up the addNode/addChild/addChildNoQuotaCheck methods in
|
HDFS-4209. Clean up the addNode/addChild/addChildNoQuotaCheck methods in
|
||||||
FSDirectory and INodeDirectory. (szetszwo)
|
FSDirectory and INodeDirectory. (szetszwo)
|
||||||
|
|
||||||
|
HDFS-3358. Specify explicitly that the NN UI status total is talking
|
||||||
|
of persistent objects on heap. (harsh)
|
||||||
|
|
||||||
OPTIMIZATIONS
|
OPTIMIZATIONS
|
||||||
|
|
||||||
BUG FIXES
|
BUG FIXES
|
||||||
|
@ -646,6 +649,8 @@ Release 2.0.3-alpha - Unreleased
|
||||||
of it is undefined after the iteration or modifications of the map.
|
of it is undefined after the iteration or modifications of the map.
|
||||||
(szetszwo)
|
(szetszwo)
|
||||||
|
|
||||||
|
HDFS-4231. BackupNode: Introduce BackupState. (shv)
|
||||||
|
|
||||||
Release 2.0.2-alpha - 2012-09-07
|
Release 2.0.2-alpha - 2012-09-07
|
||||||
|
|
||||||
INCOMPATIBLE CHANGES
|
INCOMPATIBLE CHANGES
|
||||||
|
@ -2035,6 +2040,11 @@ Release 0.23.6 - UNRELEASED
|
||||||
|
|
||||||
BUG FIXES
|
BUG FIXES
|
||||||
|
|
||||||
|
HDFS-4247. saveNamespace should be tolerant of dangling lease (daryn)
|
||||||
|
|
||||||
|
HDFS-4248. Renaming directories may incorrectly remove the paths in leases
|
||||||
|
under the tree. (daryn via szetszwo)
|
||||||
|
|
||||||
Release 0.23.5 - UNRELEASED
|
Release 0.23.5 - UNRELEASED
|
||||||
|
|
||||||
INCOMPATIBLE CHANGES
|
INCOMPATIBLE CHANGES
|
||||||
|
|
|
@ -24,6 +24,7 @@ import java.net.SocketTimeoutException;
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||||
|
import org.apache.hadoop.ha.ServiceFailedException;
|
||||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
import org.apache.hadoop.hdfs.DFSUtil;
|
import org.apache.hadoop.hdfs.DFSUtil;
|
||||||
import org.apache.hadoop.hdfs.NameNodeProxies;
|
import org.apache.hadoop.hdfs.NameNodeProxies;
|
||||||
|
@ -35,6 +36,7 @@ import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB;
|
||||||
import org.apache.hadoop.hdfs.protocolPB.JournalProtocolServerSideTranslatorPB;
|
import org.apache.hadoop.hdfs.protocolPB.JournalProtocolServerSideTranslatorPB;
|
||||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
|
||||||
import org.apache.hadoop.hdfs.server.common.Storage;
|
import org.apache.hadoop.hdfs.server.common.Storage;
|
||||||
|
import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
|
||||||
import org.apache.hadoop.hdfs.server.protocol.FenceResponse;
|
import org.apache.hadoop.hdfs.server.protocol.FenceResponse;
|
||||||
import org.apache.hadoop.hdfs.server.protocol.JournalInfo;
|
import org.apache.hadoop.hdfs.server.protocol.JournalInfo;
|
||||||
import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
|
import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
|
||||||
|
@ -414,14 +416,23 @@ public class BackupNode extends NameNode {
|
||||||
+ HdfsConstants.LAYOUT_VERSION + " actual "+ nsInfo.getLayoutVersion();
|
+ HdfsConstants.LAYOUT_VERSION + " actual "+ nsInfo.getLayoutVersion();
|
||||||
return nsInfo;
|
return nsInfo;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
protected String getNameServiceId(Configuration conf) {
|
||||||
|
return DFSUtil.getBackupNameServiceId(conf);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected HAState createHAState() {
|
||||||
|
return new BackupState();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // NameNode
|
||||||
protected NameNodeHAContext createHAContext() {
|
protected NameNodeHAContext createHAContext() {
|
||||||
return new BNHAContext();
|
return new BNHAContext();
|
||||||
}
|
}
|
||||||
|
|
||||||
private class BNHAContext extends NameNodeHAContext {
|
private class BNHAContext extends NameNodeHAContext {
|
||||||
@Override // NameNode
|
@Override // NameNodeHAContext
|
||||||
public void checkOperation(OperationCategory op)
|
public void checkOperation(OperationCategory op)
|
||||||
throws StandbyException {
|
throws StandbyException {
|
||||||
if (op == OperationCategory.UNCHECKED ||
|
if (op == OperationCategory.UNCHECKED ||
|
||||||
|
@ -435,10 +446,42 @@ public class BackupNode extends NameNode {
|
||||||
throw new StandbyException(msg);
|
throw new StandbyException(msg);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
@Override // NameNodeHAContext
|
||||||
@Override
|
public void prepareToStopStandbyServices() throws ServiceFailedException {
|
||||||
protected String getNameServiceId(Configuration conf) {
|
}
|
||||||
return DFSUtil.getBackupNameServiceId(conf);
|
|
||||||
|
/**
|
||||||
|
* Start services for BackupNode.
|
||||||
|
* <p>
|
||||||
|
* The following services should be muted
|
||||||
|
* (not run or not pass any control commands to DataNodes)
|
||||||
|
* on BackupNode:
|
||||||
|
* {@link LeaseManager.Monitor} protected by SafeMode.
|
||||||
|
* {@link BlockManager.ReplicationMonitor} protected by SafeMode.
|
||||||
|
* {@link HeartbeatManager.Monitor} protected by SafeMode.
|
||||||
|
* {@link DecommissionManager.Monitor} need to prohibit refreshNodes().
|
||||||
|
* {@link PendingReplicationBlocks.PendingReplicationMonitor} harmless,
|
||||||
|
* because ReplicationMonitor is muted.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void startActiveServices() throws IOException {
|
||||||
|
try {
|
||||||
|
namesystem.startActiveServices();
|
||||||
|
} catch (Throwable t) {
|
||||||
|
doImmediateShutdown(t);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void stopActiveServices() throws IOException {
|
||||||
|
try {
|
||||||
|
if (namesystem != null) {
|
||||||
|
namesystem.stopActiveServices();
|
||||||
|
}
|
||||||
|
} catch (Throwable t) {
|
||||||
|
doImmediateShutdown(t);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,51 @@
|
||||||
|
package org.apache.hadoop.hdfs.server.namenode;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
|
||||||
|
import org.apache.hadoop.ha.ServiceFailedException;
|
||||||
|
import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
|
||||||
|
import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
|
||||||
|
import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
|
||||||
|
import org.apache.hadoop.ipc.StandbyException;
|
||||||
|
|
||||||
|
public class BackupState extends HAState {
|
||||||
|
|
||||||
|
public BackupState() {
|
||||||
|
super(HAServiceState.STANDBY);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // HAState
|
||||||
|
public void checkOperation(HAContext context, OperationCategory op)
|
||||||
|
throws StandbyException {
|
||||||
|
context.checkOperation(op);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override // HAState
|
||||||
|
public boolean shouldPopulateReplQueues() {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void enterState(HAContext context) throws ServiceFailedException {
|
||||||
|
try {
|
||||||
|
context.startActiveServices();
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new ServiceFailedException("Failed to start backup services", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void exitState(HAContext context) throws ServiceFailedException {
|
||||||
|
try {
|
||||||
|
context.stopActiveServices();
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new ServiceFailedException("Failed to stop backup services", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void prepareToExitState(HAContext context) throws ServiceFailedException {
|
||||||
|
context.prepareToStopStandbyServices();
|
||||||
|
}
|
||||||
|
}
|
|
@ -383,7 +383,7 @@ public class FSDirectory implements Closeable {
|
||||||
writeLock();
|
writeLock();
|
||||||
try {
|
try {
|
||||||
// file is closed
|
// file is closed
|
||||||
file.setModificationTimeForce(now);
|
file.setModificationTime(now);
|
||||||
fsImage.getEditLog().logCloseFile(path, file);
|
fsImage.getEditLog().logCloseFile(path, file);
|
||||||
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
||||||
NameNode.stateChangeLog.debug("DIR* FSDirectory.closeFile: "
|
NameNode.stateChangeLog.debug("DIR* FSDirectory.closeFile: "
|
||||||
|
@ -585,8 +585,10 @@ public class FSDirectory implements Closeable {
|
||||||
+ src + " is renamed to " + dst);
|
+ src + " is renamed to " + dst);
|
||||||
}
|
}
|
||||||
// update modification time of dst and the parent of src
|
// update modification time of dst and the parent of src
|
||||||
srcInodes[srcInodes.length-2].setModificationTime(timestamp);
|
srcInodes[srcInodes.length-2].updateModificationTime(timestamp);
|
||||||
dstInodes[dstInodes.length-2].setModificationTime(timestamp);
|
dstInodes[dstInodes.length-2].updateModificationTime(timestamp);
|
||||||
|
// update moved leases with new filename
|
||||||
|
getFSNamesystem().unprotectedChangeLease(src, dst);
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
|
@ -750,8 +752,10 @@ public class FSDirectory implements Closeable {
|
||||||
"DIR* FSDirectory.unprotectedRenameTo: " + src
|
"DIR* FSDirectory.unprotectedRenameTo: " + src
|
||||||
+ " is renamed to " + dst);
|
+ " is renamed to " + dst);
|
||||||
}
|
}
|
||||||
srcInodes[srcInodes.length - 2].setModificationTime(timestamp);
|
srcInodes[srcInodes.length - 2].updateModificationTime(timestamp);
|
||||||
dstInodes[dstInodes.length - 2].setModificationTime(timestamp);
|
dstInodes[dstInodes.length - 2].updateModificationTime(timestamp);
|
||||||
|
// update moved lease with new filename
|
||||||
|
getFSNamesystem().unprotectedChangeLease(src, dst);
|
||||||
|
|
||||||
// Collect the blocks and remove the lease for previous dst
|
// Collect the blocks and remove the lease for previous dst
|
||||||
int filesDeleted = 0;
|
int filesDeleted = 0;
|
||||||
|
@ -986,12 +990,12 @@ public class FSDirectory implements Closeable {
|
||||||
if(nodeToRemove == null) continue;
|
if(nodeToRemove == null) continue;
|
||||||
|
|
||||||
nodeToRemove.setBlocks(null);
|
nodeToRemove.setBlocks(null);
|
||||||
trgParent.removeChild(nodeToRemove);
|
trgParent.removeChild(nodeToRemove, trgINodesInPath.getLatestSnapshot());
|
||||||
count++;
|
count++;
|
||||||
}
|
}
|
||||||
|
|
||||||
trgInode.setModificationTimeForce(timestamp);
|
trgInode.setModificationTime(timestamp);
|
||||||
trgParent.setModificationTime(timestamp);
|
trgParent.updateModificationTime(timestamp);
|
||||||
// update quota on the parent directory ('count' files removed, 0 space)
|
// update quota on the parent directory ('count' files removed, 0 space)
|
||||||
unprotectedUpdateCount(trgINodesInPath, trgINodes.length-1, -count, 0);
|
unprotectedUpdateCount(trgINodesInPath, trgINodes.length-1, -count, 0);
|
||||||
}
|
}
|
||||||
|
@ -1129,7 +1133,7 @@ public class FSDirectory implements Closeable {
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
// set the parent's modification time
|
// set the parent's modification time
|
||||||
inodes[inodes.length - 2].setModificationTime(mtime);
|
inodes[inodes.length - 2].updateModificationTime(mtime);
|
||||||
int filesRemoved = targetNode.collectSubtreeBlocksAndClear(collectedBlocks);
|
int filesRemoved = targetNode.collectSubtreeBlocksAndClear(collectedBlocks);
|
||||||
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
||||||
NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedDelete: "
|
NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedDelete: "
|
||||||
|
@ -1165,10 +1169,10 @@ public class FSDirectory implements Closeable {
|
||||||
/**
|
/**
|
||||||
* Replaces the specified INode.
|
* Replaces the specified INode.
|
||||||
*/
|
*/
|
||||||
private void replaceINodeUnsynced(String path, INode oldnode, INode newnode
|
private void replaceINodeUnsynced(String path, INode oldnode, INode newnode,
|
||||||
) throws IOException {
|
Snapshot latestSnapshot) throws IOException {
|
||||||
//remove the old node from the namespace
|
//remove the old node from the namespace
|
||||||
if (!oldnode.removeNode()) {
|
if (!oldnode.removeNode(latestSnapshot)) {
|
||||||
final String mess = "FSDirectory.replaceINodeUnsynced: failed to remove "
|
final String mess = "FSDirectory.replaceINodeUnsynced: failed to remove "
|
||||||
+ path;
|
+ path;
|
||||||
NameNode.stateChangeLog.warn("DIR* " + mess);
|
NameNode.stateChangeLog.warn("DIR* " + mess);
|
||||||
|
@ -1183,10 +1187,10 @@ public class FSDirectory implements Closeable {
|
||||||
* Replaces the specified INodeDirectory.
|
* Replaces the specified INodeDirectory.
|
||||||
*/
|
*/
|
||||||
public void replaceINodeDirectory(String path, INodeDirectory oldnode,
|
public void replaceINodeDirectory(String path, INodeDirectory oldnode,
|
||||||
INodeDirectory newnode) throws IOException {
|
INodeDirectory newnode, Snapshot latestSnapshot) throws IOException {
|
||||||
writeLock();
|
writeLock();
|
||||||
try {
|
try {
|
||||||
replaceINodeUnsynced(path, oldnode, newnode);
|
replaceINodeUnsynced(path, oldnode, newnode, latestSnapshot);
|
||||||
|
|
||||||
//update children's parent directory
|
//update children's parent directory
|
||||||
for(INode i : newnode.getChildrenList(null)) {
|
for(INode i : newnode.getChildrenList(null)) {
|
||||||
|
@ -1200,11 +1204,11 @@ public class FSDirectory implements Closeable {
|
||||||
/**
|
/**
|
||||||
* Replaces the specified INodeFile with the specified one.
|
* Replaces the specified INodeFile with the specified one.
|
||||||
*/
|
*/
|
||||||
public void replaceNode(String path, INodeFile oldnode, INodeFile newnode
|
public void replaceNode(String path, INodeFile oldnode, INodeFile newnode,
|
||||||
) throws IOException {
|
Snapshot latestSnapshot) throws IOException {
|
||||||
writeLock();
|
writeLock();
|
||||||
try {
|
try {
|
||||||
replaceINodeUnsynced(path, oldnode, newnode);
|
replaceINodeUnsynced(path, oldnode, newnode, latestSnapshot);
|
||||||
|
|
||||||
//Currently, oldnode and newnode are assumed to contain the same blocks.
|
//Currently, oldnode and newnode are assumed to contain the same blocks.
|
||||||
//Otherwise, blocks need to be removed from the blocksMap.
|
//Otherwise, blocks need to be removed from the blocksMap.
|
||||||
|
@ -1273,13 +1277,9 @@ public class FSDirectory implements Closeable {
|
||||||
String srcs = normalizePath(src);
|
String srcs = normalizePath(src);
|
||||||
readLock();
|
readLock();
|
||||||
try {
|
try {
|
||||||
INode targetNode = rootDir.getNode(srcs, resolveLink);
|
final INodesInPath inodesInPath = rootDir.getINodesInPath(srcs, resolveLink);
|
||||||
if (targetNode == null) {
|
final INode i = inodesInPath.getINode(0);
|
||||||
return null;
|
return i == null? null: createFileStatus(HdfsFileStatus.EMPTY_NAME, i);
|
||||||
}
|
|
||||||
else {
|
|
||||||
return createFileStatus(HdfsFileStatus.EMPTY_NAME, targetNode);
|
|
||||||
}
|
|
||||||
} finally {
|
} finally {
|
||||||
readUnlock();
|
readUnlock();
|
||||||
}
|
}
|
||||||
|
@ -1303,9 +1303,16 @@ public class FSDirectory implements Closeable {
|
||||||
* Get {@link INode} associated with the file / directory.
|
* Get {@link INode} associated with the file / directory.
|
||||||
*/
|
*/
|
||||||
public INode getINode(String src) throws UnresolvedLinkException {
|
public INode getINode(String src) throws UnresolvedLinkException {
|
||||||
|
return getINodesInPath(src).getINode(0);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get {@link INode} associated with the file / directory.
|
||||||
|
*/
|
||||||
|
public INodesInPath getINodesInPath(String src) throws UnresolvedLinkException {
|
||||||
readLock();
|
readLock();
|
||||||
try {
|
try {
|
||||||
return rootDir.getNode(src, true);
|
return rootDir.getINodesInPath(src, true);
|
||||||
} finally {
|
} finally {
|
||||||
readUnlock();
|
readUnlock();
|
||||||
}
|
}
|
||||||
|
@ -1800,7 +1807,8 @@ public class FSDirectory implements Closeable {
|
||||||
if (inodes[pos-1] == null) {
|
if (inodes[pos-1] == null) {
|
||||||
throw new NullPointerException("Panic: parent does not exist");
|
throw new NullPointerException("Panic: parent does not exist");
|
||||||
}
|
}
|
||||||
final boolean added = ((INodeDirectory)inodes[pos-1]).addChild(child, true);
|
final boolean added = ((INodeDirectory)inodes[pos-1]).addChild(child, true,
|
||||||
|
inodesInPath.getLatestSnapshot());
|
||||||
if (!added) {
|
if (!added) {
|
||||||
updateCount(inodesInPath, pos, -counts.getNsCount(), -counts.getDsCount(), true);
|
updateCount(inodesInPath, pos, -counts.getNsCount(), -counts.getDsCount(), true);
|
||||||
}
|
}
|
||||||
|
@ -1824,7 +1832,8 @@ public class FSDirectory implements Closeable {
|
||||||
private INode removeLastINode(final INodesInPath inodesInPath) {
|
private INode removeLastINode(final INodesInPath inodesInPath) {
|
||||||
final INode[] inodes = inodesInPath.getINodes();
|
final INode[] inodes = inodesInPath.getINodes();
|
||||||
final int pos = inodes.length - 1;
|
final int pos = inodes.length - 1;
|
||||||
INode removedNode = ((INodeDirectory)inodes[pos-1]).removeChild(inodes[pos]);
|
INode removedNode = ((INodeDirectory)inodes[pos-1]).removeChild(inodes[pos],
|
||||||
|
inodesInPath.getLatestSnapshot());
|
||||||
if (removedNode != null) {
|
if (removedNode != null) {
|
||||||
INode.DirCounts counts = new INode.DirCounts();
|
INode.DirCounts counts = new INode.DirCounts();
|
||||||
removedNode.spaceConsumedInTree(counts);
|
removedNode.spaceConsumedInTree(counts);
|
||||||
|
@ -1965,8 +1974,8 @@ public class FSDirectory implements Closeable {
|
||||||
}
|
}
|
||||||
|
|
||||||
String srcs = normalizePath(src);
|
String srcs = normalizePath(src);
|
||||||
final INode[] inodes = rootDir.getMutableINodesInPath(srcs, true)
|
final INodesInPath inodesInPath = rootDir.getMutableINodesInPath(srcs, true);
|
||||||
.getINodes();
|
final INode[] inodes = inodesInPath.getINodes();
|
||||||
INodeDirectory dirNode = INodeDirectory.valueOf(inodes[inodes.length-1], srcs);
|
INodeDirectory dirNode = INodeDirectory.valueOf(inodes[inodes.length-1], srcs);
|
||||||
if (dirNode.isRoot() && nsQuota == HdfsConstants.QUOTA_RESET) {
|
if (dirNode.isRoot() && nsQuota == HdfsConstants.QUOTA_RESET) {
|
||||||
throw new IllegalArgumentException("Cannot clear namespace quota on root.");
|
throw new IllegalArgumentException("Cannot clear namespace quota on root.");
|
||||||
|
@ -1988,7 +1997,7 @@ public class FSDirectory implements Closeable {
|
||||||
INodeDirectory newNode = new INodeDirectory(dirNode);
|
INodeDirectory newNode = new INodeDirectory(dirNode);
|
||||||
INodeDirectory parent = (INodeDirectory)inodes[inodes.length-2];
|
INodeDirectory parent = (INodeDirectory)inodes[inodes.length-2];
|
||||||
dirNode = newNode;
|
dirNode = newNode;
|
||||||
parent.replaceChild(newNode);
|
parent.replaceChild(newNode, inodesInPath.getLatestSnapshot());
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
// a non-quota directory; so replace it with a directory with quota
|
// a non-quota directory; so replace it with a directory with quota
|
||||||
|
@ -1997,7 +2006,7 @@ public class FSDirectory implements Closeable {
|
||||||
// non-root directory node; parent != null
|
// non-root directory node; parent != null
|
||||||
INodeDirectory parent = (INodeDirectory)inodes[inodes.length-2];
|
INodeDirectory parent = (INodeDirectory)inodes[inodes.length-2];
|
||||||
dirNode = newNode;
|
dirNode = newNode;
|
||||||
parent.replaceChild(newNode);
|
parent.replaceChild(newNode, inodesInPath.getLatestSnapshot());
|
||||||
}
|
}
|
||||||
return (oldNsQuota != nsQuota || oldDsQuota != dsQuota) ? dirNode : null;
|
return (oldNsQuota != nsQuota || oldDsQuota != dsQuota) ? dirNode : null;
|
||||||
}
|
}
|
||||||
|
@ -2061,7 +2070,7 @@ public class FSDirectory implements Closeable {
|
||||||
assert hasWriteLock();
|
assert hasWriteLock();
|
||||||
boolean status = false;
|
boolean status = false;
|
||||||
if (mtime != -1) {
|
if (mtime != -1) {
|
||||||
inode.setModificationTimeForce(mtime);
|
inode.setModificationTime(mtime);
|
||||||
status = true;
|
status = true;
|
||||||
}
|
}
|
||||||
if (atime != -1) {
|
if (atime != -1) {
|
||||||
|
|
|
@ -31,7 +31,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.classification.InterfaceStability;
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.hdfs.protocol.Block;
|
import org.apache.hadoop.hdfs.protocol.Block;
|
||||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
||||||
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
||||||
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
|
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
|
||||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
||||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
|
||||||
|
@ -58,6 +57,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SymlinkOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateBlocksOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateBlocksOp;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
|
||||||
|
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory.INodesInPath;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
|
import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
|
||||||
import org.apache.hadoop.hdfs.util.Holder;
|
import org.apache.hadoop.hdfs.util.Holder;
|
||||||
|
|
||||||
|
@ -246,7 +246,8 @@ public class FSEditLogLoader {
|
||||||
// 3. OP_ADD to open file for append
|
// 3. OP_ADD to open file for append
|
||||||
|
|
||||||
// See if the file already exists (persistBlocks call)
|
// See if the file already exists (persistBlocks call)
|
||||||
INodeFile oldFile = getINodeFile(fsDir, addCloseOp.path);
|
final INodesInPath inodesInPath = fsDir.getINodesInPath(addCloseOp.path);
|
||||||
|
INodeFile oldFile = toINodeFile(inodesInPath.getINode(0), addCloseOp.path);
|
||||||
INodeFile newFile = oldFile;
|
INodeFile newFile = oldFile;
|
||||||
if (oldFile == null) { // this is OP_ADD on a new file (case 1)
|
if (oldFile == null) { // this is OP_ADD on a new file (case 1)
|
||||||
// versions > 0 support per file replication
|
// versions > 0 support per file replication
|
||||||
|
@ -272,7 +273,7 @@ public class FSEditLogLoader {
|
||||||
}
|
}
|
||||||
fsNamesys.prepareFileForWrite(addCloseOp.path, oldFile,
|
fsNamesys.prepareFileForWrite(addCloseOp.path, oldFile,
|
||||||
addCloseOp.clientName, addCloseOp.clientMachine, null,
|
addCloseOp.clientName, addCloseOp.clientMachine, null,
|
||||||
false);
|
false, inodesInPath.getLatestSnapshot());
|
||||||
newFile = getINodeFile(fsDir, addCloseOp.path);
|
newFile = getINodeFile(fsDir, addCloseOp.path);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -282,7 +283,7 @@ public class FSEditLogLoader {
|
||||||
|
|
||||||
// Update the salient file attributes.
|
// Update the salient file attributes.
|
||||||
newFile.setAccessTime(addCloseOp.atime);
|
newFile.setAccessTime(addCloseOp.atime);
|
||||||
newFile.setModificationTimeForce(addCloseOp.mtime);
|
newFile.setModificationTime(addCloseOp.mtime);
|
||||||
updateBlocks(fsDir, addCloseOp, newFile);
|
updateBlocks(fsDir, addCloseOp, newFile);
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
@ -296,7 +297,8 @@ public class FSEditLogLoader {
|
||||||
" clientMachine " + addCloseOp.clientMachine);
|
" clientMachine " + addCloseOp.clientMachine);
|
||||||
}
|
}
|
||||||
|
|
||||||
INodeFile oldFile = getINodeFile(fsDir, addCloseOp.path);
|
final INodesInPath inodesInPath = fsDir.getINodesInPath(addCloseOp.path);
|
||||||
|
INodeFile oldFile = toINodeFile(inodesInPath.getINode(0), addCloseOp.path);
|
||||||
if (oldFile == null) {
|
if (oldFile == null) {
|
||||||
throw new IOException("Operation trying to close non-existent file " +
|
throw new IOException("Operation trying to close non-existent file " +
|
||||||
addCloseOp.path);
|
addCloseOp.path);
|
||||||
|
@ -304,7 +306,7 @@ public class FSEditLogLoader {
|
||||||
|
|
||||||
// Update the salient file attributes.
|
// Update the salient file attributes.
|
||||||
oldFile.setAccessTime(addCloseOp.atime);
|
oldFile.setAccessTime(addCloseOp.atime);
|
||||||
oldFile.setModificationTimeForce(addCloseOp.mtime);
|
oldFile.setModificationTime(addCloseOp.mtime);
|
||||||
updateBlocks(fsDir, addCloseOp, oldFile);
|
updateBlocks(fsDir, addCloseOp, oldFile);
|
||||||
|
|
||||||
// Now close the file
|
// Now close the file
|
||||||
|
@ -322,7 +324,8 @@ public class FSEditLogLoader {
|
||||||
INodeFileUnderConstruction ucFile = (INodeFileUnderConstruction) oldFile;
|
INodeFileUnderConstruction ucFile = (INodeFileUnderConstruction) oldFile;
|
||||||
fsNamesys.leaseManager.removeLeaseWithPrefixPath(addCloseOp.path);
|
fsNamesys.leaseManager.removeLeaseWithPrefixPath(addCloseOp.path);
|
||||||
INodeFile newFile = ucFile.convertToInodeFile();
|
INodeFile newFile = ucFile.convertToInodeFile();
|
||||||
fsDir.replaceNode(addCloseOp.path, ucFile, newFile);
|
fsDir.replaceNode(addCloseOp.path, ucFile, newFile,
|
||||||
|
inodesInPath.getLatestSnapshot());
|
||||||
}
|
}
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
@ -360,10 +363,8 @@ public class FSEditLogLoader {
|
||||||
}
|
}
|
||||||
case OP_RENAME_OLD: {
|
case OP_RENAME_OLD: {
|
||||||
RenameOldOp renameOp = (RenameOldOp)op;
|
RenameOldOp renameOp = (RenameOldOp)op;
|
||||||
HdfsFileStatus dinfo = fsDir.getFileInfo(renameOp.dst, false);
|
|
||||||
fsDir.unprotectedRenameTo(renameOp.src, renameOp.dst,
|
fsDir.unprotectedRenameTo(renameOp.src, renameOp.dst,
|
||||||
renameOp.timestamp);
|
renameOp.timestamp);
|
||||||
fsNamesys.unprotectedChangeLease(renameOp.src, renameOp.dst, dinfo);
|
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
case OP_DELETE: {
|
case OP_DELETE: {
|
||||||
|
@ -433,11 +434,8 @@ public class FSEditLogLoader {
|
||||||
}
|
}
|
||||||
case OP_RENAME: {
|
case OP_RENAME: {
|
||||||
RenameOp renameOp = (RenameOp)op;
|
RenameOp renameOp = (RenameOp)op;
|
||||||
|
|
||||||
HdfsFileStatus dinfo = fsDir.getFileInfo(renameOp.dst, false);
|
|
||||||
fsDir.unprotectedRenameTo(renameOp.src, renameOp.dst,
|
fsDir.unprotectedRenameTo(renameOp.src, renameOp.dst,
|
||||||
renameOp.timestamp, renameOp.options);
|
renameOp.timestamp, renameOp.options);
|
||||||
fsNamesys.unprotectedChangeLease(renameOp.src, renameOp.dst, dinfo);
|
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
case OP_GET_DELEGATION_TOKEN: {
|
case OP_GET_DELEGATION_TOKEN: {
|
||||||
|
@ -512,7 +510,11 @@ public class FSEditLogLoader {
|
||||||
|
|
||||||
private static INodeFile getINodeFile(FSDirectory fsDir, String path)
|
private static INodeFile getINodeFile(FSDirectory fsDir, String path)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
INode inode = fsDir.getINode(path);
|
return toINodeFile(fsDir.getINode(path), path);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static INodeFile toINodeFile(INode inode, String path)
|
||||||
|
throws IOException {
|
||||||
if (inode != null) {
|
if (inode != null) {
|
||||||
if (!(inode instanceof INodeFile)) {
|
if (!(inode instanceof INodeFile)) {
|
||||||
throw new IOException("Operation trying to get non-file " + path);
|
throw new IOException("Operation trying to get non-file " + path);
|
||||||
|
|
|
@ -44,6 +44,7 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
|
||||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
||||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
||||||
import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
|
import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
|
||||||
|
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory.INodesInPath;
|
||||||
import org.apache.hadoop.hdfs.util.ReadOnlyList;
|
import org.apache.hadoop.hdfs.util.ReadOnlyList;
|
||||||
import org.apache.hadoop.io.MD5Hash;
|
import org.apache.hadoop.io.MD5Hash;
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
|
@ -202,7 +203,7 @@ class FSImageFormat {
|
||||||
if (nsQuota != -1 || dsQuota != -1) {
|
if (nsQuota != -1 || dsQuota != -1) {
|
||||||
fsDir.rootDir.setQuota(nsQuota, dsQuota);
|
fsDir.rootDir.setQuota(nsQuota, dsQuota);
|
||||||
}
|
}
|
||||||
fsDir.rootDir.setModificationTime(root.getModificationTime());
|
fsDir.rootDir.cloneModificationTime(root);
|
||||||
fsDir.rootDir.clonePermissionStatus(root);
|
fsDir.rootDir.clonePermissionStatus(root);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -305,7 +306,7 @@ class FSImageFormat {
|
||||||
*/
|
*/
|
||||||
void addToParent(INodeDirectory parent, INode child) {
|
void addToParent(INodeDirectory parent, INode child) {
|
||||||
// NOTE: This does not update space counts for parents
|
// NOTE: This does not update space counts for parents
|
||||||
if (!parent.addChild(child, false)) {
|
if (!parent.addChild(child, false, null)) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
namesystem.dir.cacheName(child);
|
namesystem.dir.cacheName(child);
|
||||||
|
@ -388,8 +389,9 @@ class FSImageFormat {
|
||||||
|
|
||||||
// verify that file exists in namespace
|
// verify that file exists in namespace
|
||||||
String path = cons.getLocalName();
|
String path = cons.getLocalName();
|
||||||
INodeFile oldnode = INodeFile.valueOf(fsDir.getINode(path), path);
|
final INodesInPath inodesInPath = fsDir.getINodesInPath(path);
|
||||||
fsDir.replaceNode(path, oldnode, cons);
|
INodeFile oldnode = INodeFile.valueOf(inodesInPath.getINode(0), path);
|
||||||
|
fsDir.replaceNode(path, oldnode, cons, inodesInPath.getLatestSnapshot());
|
||||||
namesystem.leaseManager.addLease(cons.getClientName(), path);
|
namesystem.leaseManager.addLease(cons.getClientName(), path);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -121,6 +121,7 @@ import org.apache.hadoop.fs.UnresolvedLinkException;
|
||||||
import org.apache.hadoop.fs.permission.FsAction;
|
import org.apache.hadoop.fs.permission.FsAction;
|
||||||
import org.apache.hadoop.fs.permission.FsPermission;
|
import org.apache.hadoop.fs.permission.FsPermission;
|
||||||
import org.apache.hadoop.fs.permission.PermissionStatus;
|
import org.apache.hadoop.fs.permission.PermissionStatus;
|
||||||
|
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
|
||||||
import org.apache.hadoop.ha.ServiceFailedException;
|
import org.apache.hadoop.ha.ServiceFailedException;
|
||||||
import org.apache.hadoop.hdfs.DFSUtil;
|
import org.apache.hadoop.hdfs.DFSUtil;
|
||||||
import org.apache.hadoop.hdfs.HAUtil;
|
import org.apache.hadoop.hdfs.HAUtil;
|
||||||
|
@ -165,7 +166,6 @@ import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory.INodesInPath;
|
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory.INodesInPath;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
|
import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
|
import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.ha.ActiveState;
|
|
||||||
import org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer;
|
import org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
|
import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
|
import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
|
||||||
|
@ -173,7 +173,9 @@ import org.apache.hadoop.hdfs.server.namenode.ha.StandbyCheckpointer;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.ha.StandbyState;
|
import org.apache.hadoop.hdfs.server.namenode.ha.StandbyState;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
|
import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
|
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
|
||||||
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithLink;
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithLink;
|
||||||
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager;
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
|
import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
|
||||||
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
|
||||||
|
@ -1036,7 +1038,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
||||||
long totalInodes = this.dir.totalInodes();
|
long totalInodes = this.dir.totalInodes();
|
||||||
long totalBlocks = this.getBlocksTotal();
|
long totalBlocks = this.getBlocksTotal();
|
||||||
out.println(totalInodes + " files and directories, " + totalBlocks
|
out.println(totalInodes + " files and directories, " + totalBlocks
|
||||||
+ " blocks = " + (totalInodes + totalBlocks) + " total");
|
+ " blocks = " + (totalInodes + totalBlocks)
|
||||||
|
+ " total filesystem objects");
|
||||||
|
|
||||||
blockManager.metaSave(out);
|
blockManager.metaSave(out);
|
||||||
}
|
}
|
||||||
|
@ -1820,7 +1823,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
||||||
try {
|
try {
|
||||||
blockManager.verifyReplication(src, replication, clientMachine);
|
blockManager.verifyReplication(src, replication, clientMachine);
|
||||||
boolean create = flag.contains(CreateFlag.CREATE);
|
boolean create = flag.contains(CreateFlag.CREATE);
|
||||||
final INode myFile = dir.getINode(src);
|
|
||||||
|
final INodesInPath inodesInPath = dir.getINodesInPath(src);
|
||||||
|
final INode myFile = inodesInPath.getINode(0);
|
||||||
if (myFile == null) {
|
if (myFile == null) {
|
||||||
if (!create) {
|
if (!create) {
|
||||||
throw new FileNotFoundException("failed to overwrite or append to non-existent file "
|
throw new FileNotFoundException("failed to overwrite or append to non-existent file "
|
||||||
|
@ -1847,8 +1852,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
||||||
|
|
||||||
if (append && myFile != null) {
|
if (append && myFile != null) {
|
||||||
final INodeFile f = INodeFile.valueOf(myFile, src);
|
final INodeFile f = INodeFile.valueOf(myFile, src);
|
||||||
return prepareFileForWrite(
|
return prepareFileForWrite(src, f, holder, clientMachine, clientNode,
|
||||||
src, f, holder, clientMachine, clientNode, true);
|
true, inodesInPath.getLatestSnapshot());
|
||||||
} else {
|
} else {
|
||||||
// Now we can add the name to the filesystem. This file has no
|
// Now we can add the name to the filesystem. This file has no
|
||||||
// blocks associated with it.
|
// blocks associated with it.
|
||||||
|
@ -1896,7 +1901,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
||||||
*/
|
*/
|
||||||
LocatedBlock prepareFileForWrite(String src, INodeFile file,
|
LocatedBlock prepareFileForWrite(String src, INodeFile file,
|
||||||
String leaseHolder, String clientMachine, DatanodeDescriptor clientNode,
|
String leaseHolder, String clientMachine, DatanodeDescriptor clientNode,
|
||||||
boolean writeToEditLog) throws IOException {
|
boolean writeToEditLog, Snapshot latestSnapshot) throws IOException {
|
||||||
//TODO SNAPSHOT: INodeFileUnderConstruction with link
|
//TODO SNAPSHOT: INodeFileUnderConstruction with link
|
||||||
INodeFileUnderConstruction cons = new INodeFileUnderConstruction(
|
INodeFileUnderConstruction cons = new INodeFileUnderConstruction(
|
||||||
file.getLocalNameBytes(),
|
file.getLocalNameBytes(),
|
||||||
|
@ -1908,7 +1913,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
||||||
leaseHolder,
|
leaseHolder,
|
||||||
clientMachine,
|
clientMachine,
|
||||||
clientNode);
|
clientNode);
|
||||||
dir.replaceNode(src, file, cons);
|
dir.replaceNode(src, file, cons, latestSnapshot);
|
||||||
leaseManager.addLease(cons.getClientName(), src);
|
leaseManager.addLease(cons.getClientName(), src);
|
||||||
|
|
||||||
LocatedBlock ret = blockManager.convertLastBlockToUnderConstruction(cons);
|
LocatedBlock ret = blockManager.convertLastBlockToUnderConstruction(cons);
|
||||||
|
@ -2158,7 +2163,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
||||||
// have we exceeded the configured limit of fs objects.
|
// have we exceeded the configured limit of fs objects.
|
||||||
checkFsObjectLimit();
|
checkFsObjectLimit();
|
||||||
|
|
||||||
INodeFileUnderConstruction pendingFile = checkLease(src, clientName);
|
final INodeFileUnderConstruction pendingFile = checkLease(
|
||||||
|
src, clientName, dir.getINode(src));
|
||||||
BlockInfo lastBlockInFile = pendingFile.getLastBlock();
|
BlockInfo lastBlockInFile = pendingFile.getLastBlock();
|
||||||
if (!Block.matchingIdAndGenStamp(previousBlock, lastBlockInFile)) {
|
if (!Block.matchingIdAndGenStamp(previousBlock, lastBlockInFile)) {
|
||||||
// The block that the client claims is the current last block
|
// The block that the client claims is the current last block
|
||||||
|
@ -2294,7 +2300,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
||||||
}
|
}
|
||||||
|
|
||||||
//check lease
|
//check lease
|
||||||
final INodeFileUnderConstruction file = checkLease(src, clientName);
|
final INodeFileUnderConstruction file = checkLease(
|
||||||
|
src, clientName, dir.getINode(src));
|
||||||
clientnode = file.getClientNode();
|
clientnode = file.getClientNode();
|
||||||
preferredblocksize = file.getPreferredBlockSize();
|
preferredblocksize = file.getPreferredBlockSize();
|
||||||
|
|
||||||
|
@ -2340,7 +2347,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
||||||
throw new SafeModeException("Cannot abandon block " + b +
|
throw new SafeModeException("Cannot abandon block " + b +
|
||||||
" for fle" + src, safeMode);
|
" for fle" + src, safeMode);
|
||||||
}
|
}
|
||||||
INodeFileUnderConstruction file = checkLease(src, holder);
|
final INodesInPath inodesInPath = checkLease(src, holder);
|
||||||
|
final INodeFileUnderConstruction file
|
||||||
|
= (INodeFileUnderConstruction)inodesInPath.getINode(0);
|
||||||
dir.removeBlock(src, file, ExtendedBlock.getLocalBlock(b));
|
dir.removeBlock(src, file, ExtendedBlock.getLocalBlock(b));
|
||||||
if(NameNode.stateChangeLog.isDebugEnabled()) {
|
if(NameNode.stateChangeLog.isDebugEnabled()) {
|
||||||
NameNode.stateChangeLog.debug("BLOCK* NameSystem.abandonBlock: "
|
NameNode.stateChangeLog.debug("BLOCK* NameSystem.abandonBlock: "
|
||||||
|
@ -2357,11 +2366,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
// make sure that we still have the lease on this file.
|
/** make sure that we still have the lease on this file. */
|
||||||
private INodeFileUnderConstruction checkLease(String src, String holder)
|
private INodesInPath checkLease(String src, String holder)
|
||||||
throws LeaseExpiredException, UnresolvedLinkException {
|
throws LeaseExpiredException, UnresolvedLinkException {
|
||||||
assert hasReadOrWriteLock();
|
assert hasReadOrWriteLock();
|
||||||
return checkLease(src, holder, dir.getINode(src));
|
final INodesInPath inodesInPath = dir.getINodesInPath(src);
|
||||||
|
checkLease(src, holder, inodesInPath.getINode(0));
|
||||||
|
return inodesInPath;
|
||||||
}
|
}
|
||||||
|
|
||||||
private INodeFileUnderConstruction checkLease(String src, String holder,
|
private INodeFileUnderConstruction checkLease(String src, String holder,
|
||||||
|
@ -2424,9 +2435,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
||||||
throw new SafeModeException("Cannot complete file " + src, safeMode);
|
throw new SafeModeException("Cannot complete file " + src, safeMode);
|
||||||
}
|
}
|
||||||
|
|
||||||
INodeFileUnderConstruction pendingFile;
|
final INodesInPath inodesInPath;
|
||||||
|
final INodeFileUnderConstruction pendingFile;
|
||||||
try {
|
try {
|
||||||
pendingFile = checkLease(src, holder);
|
inodesInPath = checkLease(src, holder);
|
||||||
|
pendingFile = (INodeFileUnderConstruction)inodesInPath.getINode(0);
|
||||||
} catch (LeaseExpiredException lee) {
|
} catch (LeaseExpiredException lee) {
|
||||||
final INode inode = dir.getINode(src);
|
final INode inode = dir.getINode(src);
|
||||||
if (inode != null && inode instanceof INodeFile && !inode.isUnderConstruction()) {
|
if (inode != null && inode instanceof INodeFile && !inode.isUnderConstruction()) {
|
||||||
|
@ -2454,7 +2467,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
finalizeINodeFileUnderConstruction(src, pendingFile);
|
finalizeINodeFileUnderConstruction(src, pendingFile,
|
||||||
|
inodesInPath.getLatestSnapshot());
|
||||||
|
|
||||||
NameNode.stateChangeLog.info("DIR* completeFile: " + src + " is closed by "
|
NameNode.stateChangeLog.info("DIR* completeFile: " + src + " is closed by "
|
||||||
+ holder);
|
+ holder);
|
||||||
|
@ -2595,15 +2609,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
||||||
if (isPermissionEnabled) {
|
if (isPermissionEnabled) {
|
||||||
//We should not be doing this. This is move() not renameTo().
|
//We should not be doing this. This is move() not renameTo().
|
||||||
//but for now,
|
//but for now,
|
||||||
|
//NOTE: yes, this is bad! it's assuming much lower level behavior
|
||||||
|
// of rewriting the dst
|
||||||
String actualdst = dir.isDir(dst)?
|
String actualdst = dir.isDir(dst)?
|
||||||
dst + Path.SEPARATOR + new Path(src).getName(): dst;
|
dst + Path.SEPARATOR + new Path(src).getName(): dst;
|
||||||
checkParentAccess(src, FsAction.WRITE);
|
checkParentAccess(src, FsAction.WRITE);
|
||||||
checkAncestorAccess(actualdst, FsAction.WRITE);
|
checkAncestorAccess(actualdst, FsAction.WRITE);
|
||||||
}
|
}
|
||||||
|
|
||||||
HdfsFileStatus dinfo = dir.getFileInfo(dst, false);
|
|
||||||
if (dir.renameTo(src, dst)) {
|
if (dir.renameTo(src, dst)) {
|
||||||
unprotectedChangeLease(src, dst, dinfo); // update lease with new filename
|
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
return false;
|
return false;
|
||||||
|
@ -2654,9 +2668,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
||||||
checkAncestorAccess(dst, FsAction.WRITE);
|
checkAncestorAccess(dst, FsAction.WRITE);
|
||||||
}
|
}
|
||||||
|
|
||||||
HdfsFileStatus dinfo = dir.getFileInfo(dst, false);
|
|
||||||
dir.renameTo(src, dst, options);
|
dir.renameTo(src, dst, options);
|
||||||
unprotectedChangeLease(src, dst, dinfo); // update lease with new filename
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -3036,7 +3048,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
||||||
if (isInSafeMode()) {
|
if (isInSafeMode()) {
|
||||||
throw new SafeModeException("Cannot fsync file " + src, safeMode);
|
throw new SafeModeException("Cannot fsync file " + src, safeMode);
|
||||||
}
|
}
|
||||||
INodeFileUnderConstruction pendingFile = checkLease(src, clientName);
|
final INodeFileUnderConstruction pendingFile = checkLease(
|
||||||
|
src, clientName, dir.getINode(src));
|
||||||
if (lastBlockLength > 0) {
|
if (lastBlockLength > 0) {
|
||||||
pendingFile.updateLengthOfLastBlock(lastBlockLength);
|
pendingFile.updateLengthOfLastBlock(lastBlockLength);
|
||||||
}
|
}
|
||||||
|
@ -3068,8 +3081,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
||||||
assert !isInSafeMode();
|
assert !isInSafeMode();
|
||||||
assert hasWriteLock();
|
assert hasWriteLock();
|
||||||
|
|
||||||
|
final INodesInPath inodesInPath = dir.getINodesInPath(src);
|
||||||
final INodeFileUnderConstruction pendingFile
|
final INodeFileUnderConstruction pendingFile
|
||||||
= INodeFileUnderConstruction.valueOf(dir.getINode(src), src);
|
= INodeFileUnderConstruction.valueOf(inodesInPath.getINode(0), src);
|
||||||
int nrBlocks = pendingFile.numBlocks();
|
int nrBlocks = pendingFile.numBlocks();
|
||||||
BlockInfo[] blocks = pendingFile.getBlocks();
|
BlockInfo[] blocks = pendingFile.getBlocks();
|
||||||
|
|
||||||
|
@ -3086,7 +3100,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
||||||
// If there are no incomplete blocks associated with this file,
|
// If there are no incomplete blocks associated with this file,
|
||||||
// then reap lease immediately and close the file.
|
// then reap lease immediately and close the file.
|
||||||
if(nrCompleteBlocks == nrBlocks) {
|
if(nrCompleteBlocks == nrBlocks) {
|
||||||
finalizeINodeFileUnderConstruction(src, pendingFile);
|
finalizeINodeFileUnderConstruction(src, pendingFile,
|
||||||
|
inodesInPath.getLatestSnapshot());
|
||||||
NameNode.stateChangeLog.warn("BLOCK*"
|
NameNode.stateChangeLog.warn("BLOCK*"
|
||||||
+ " internalReleaseLease: All existing blocks are COMPLETE,"
|
+ " internalReleaseLease: All existing blocks are COMPLETE,"
|
||||||
+ " lease removed, file closed.");
|
+ " lease removed, file closed.");
|
||||||
|
@ -3134,7 +3149,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
||||||
// Close file if committed blocks are minimally replicated
|
// Close file if committed blocks are minimally replicated
|
||||||
if(penultimateBlockMinReplication &&
|
if(penultimateBlockMinReplication &&
|
||||||
blockManager.checkMinReplication(lastBlock)) {
|
blockManager.checkMinReplication(lastBlock)) {
|
||||||
finalizeINodeFileUnderConstruction(src, pendingFile);
|
finalizeINodeFileUnderConstruction(src, pendingFile,
|
||||||
|
inodesInPath.getLatestSnapshot());
|
||||||
NameNode.stateChangeLog.warn("BLOCK*"
|
NameNode.stateChangeLog.warn("BLOCK*"
|
||||||
+ " internalReleaseLease: Committed blocks are minimally replicated,"
|
+ " internalReleaseLease: Committed blocks are minimally replicated,"
|
||||||
+ " lease removed, file closed.");
|
+ " lease removed, file closed.");
|
||||||
|
@ -3212,7 +3228,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
||||||
}
|
}
|
||||||
|
|
||||||
private void finalizeINodeFileUnderConstruction(String src,
|
private void finalizeINodeFileUnderConstruction(String src,
|
||||||
INodeFileUnderConstruction pendingFile)
|
INodeFileUnderConstruction pendingFile, Snapshot latestSnapshot)
|
||||||
throws IOException, UnresolvedLinkException {
|
throws IOException, UnresolvedLinkException {
|
||||||
assert hasWriteLock();
|
assert hasWriteLock();
|
||||||
leaseManager.removeLease(pendingFile.getClientName(), src);
|
leaseManager.removeLease(pendingFile.getClientName(), src);
|
||||||
|
@ -3220,7 +3236,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
||||||
// The file is no longer pending.
|
// The file is no longer pending.
|
||||||
// Create permanent INode, update blocks
|
// Create permanent INode, update blocks
|
||||||
INodeFile newFile = pendingFile.convertToInodeFile();
|
INodeFile newFile = pendingFile.convertToInodeFile();
|
||||||
dir.replaceNode(src, pendingFile, newFile);
|
dir.replaceNode(src, pendingFile, newFile, latestSnapshot);
|
||||||
|
|
||||||
// close file and persist block allocations for this file
|
// close file and persist block allocations for this file
|
||||||
dir.closeFile(src, newFile);
|
dir.closeFile(src, newFile);
|
||||||
|
@ -3312,7 +3328,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
||||||
commitOrCompleteLastBlock(pendingFile, storedBlock);
|
commitOrCompleteLastBlock(pendingFile, storedBlock);
|
||||||
|
|
||||||
//remove lease, close file
|
//remove lease, close file
|
||||||
finalizeINodeFileUnderConstruction(src, pendingFile);
|
finalizeINodeFileUnderConstruction(src, pendingFile,
|
||||||
|
INodeDirectorySnapshottable.findLatestSnapshot(pendingFile));
|
||||||
} else {
|
} else {
|
||||||
// If this commit does not want to close the file, persist blocks
|
// If this commit does not want to close the file, persist blocks
|
||||||
dir.persistBlocks(src, pendingFile);
|
dir.persistBlocks(src, pendingFile);
|
||||||
|
@ -3483,9 +3500,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
||||||
private NNHAStatusHeartbeat createHaStatusHeartbeat() {
|
private NNHAStatusHeartbeat createHaStatusHeartbeat() {
|
||||||
HAState state = haContext.getState();
|
HAState state = haContext.getState();
|
||||||
NNHAStatusHeartbeat.State hbState;
|
NNHAStatusHeartbeat.State hbState;
|
||||||
if (state instanceof ActiveState) {
|
if (state.getServiceState() == HAServiceState.ACTIVE) {
|
||||||
hbState = NNHAStatusHeartbeat.State.ACTIVE;
|
hbState = NNHAStatusHeartbeat.State.ACTIVE;
|
||||||
} else if (state instanceof StandbyState) {
|
} else if (state.getServiceState() == HAServiceState.STANDBY) {
|
||||||
hbState = NNHAStatusHeartbeat.State.STANDBY;
|
hbState = NNHAStatusHeartbeat.State.STANDBY;
|
||||||
} else {
|
} else {
|
||||||
throw new AssertionError("Invalid state: " + state.getClass());
|
throw new AssertionError("Invalid state: " + state.getClass());
|
||||||
|
@ -4939,31 +4956,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
||||||
|
|
||||||
// rename was successful. If any part of the renamed subtree had
|
// rename was successful. If any part of the renamed subtree had
|
||||||
// files that were being written to, update with new filename.
|
// files that were being written to, update with new filename.
|
||||||
void unprotectedChangeLease(String src, String dst, HdfsFileStatus dinfo) {
|
void unprotectedChangeLease(String src, String dst) {
|
||||||
String overwrite;
|
|
||||||
String replaceBy;
|
|
||||||
assert hasWriteLock();
|
assert hasWriteLock();
|
||||||
|
leaseManager.changeLease(src, dst);
|
||||||
boolean destinationExisted = true;
|
|
||||||
if (dinfo == null) {
|
|
||||||
destinationExisted = false;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (destinationExisted && dinfo.isDir()) {
|
|
||||||
Path spath = new Path(src);
|
|
||||||
Path parent = spath.getParent();
|
|
||||||
if (parent.isRoot()) {
|
|
||||||
overwrite = parent.toString();
|
|
||||||
} else {
|
|
||||||
overwrite = parent.toString() + Path.SEPARATOR;
|
|
||||||
}
|
|
||||||
replaceBy = dst + Path.SEPARATOR;
|
|
||||||
} else {
|
|
||||||
overwrite = src;
|
|
||||||
replaceBy = dst;
|
|
||||||
}
|
|
||||||
|
|
||||||
leaseManager.changeLease(src, dst, overwrite, replaceBy);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -4974,19 +4969,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
||||||
// lock on our behalf. If we took the read lock here, we could block
|
// lock on our behalf. If we took the read lock here, we could block
|
||||||
// for fairness if a writer is waiting on the lock.
|
// for fairness if a writer is waiting on the lock.
|
||||||
synchronized (leaseManager) {
|
synchronized (leaseManager) {
|
||||||
out.writeInt(leaseManager.countPath()); // write the size
|
Map<String, INodeFileUnderConstruction> nodes =
|
||||||
|
leaseManager.getINodesUnderConstruction();
|
||||||
for (Lease lease : leaseManager.getSortedLeases()) {
|
out.writeInt(nodes.size()); // write the size
|
||||||
for(String path : lease.getPaths()) {
|
for (Map.Entry<String, INodeFileUnderConstruction> entry
|
||||||
// verify that path exists in namespace
|
: nodes.entrySet()) {
|
||||||
final INodeFileUnderConstruction cons;
|
FSImageSerialization.writeINodeUnderConstruction(
|
||||||
try {
|
out, entry.getValue(), entry.getKey());
|
||||||
cons = INodeFileUnderConstruction.valueOf(dir.getINode(path), path);
|
|
||||||
} catch (UnresolvedLinkException e) {
|
|
||||||
throw new AssertionError("Lease files should reside on this FS");
|
|
||||||
}
|
|
||||||
FSImageSerialization.writeINodeUnderConstruction(out, cons, path);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -33,6 +33,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
|
||||||
import org.apache.hadoop.hdfs.protocol.Block;
|
import org.apache.hadoop.hdfs.protocol.Block;
|
||||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
|
||||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
||||||
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
|
||||||
import org.apache.hadoop.hdfs.util.ReadOnlyList;
|
import org.apache.hadoop.hdfs.util.ReadOnlyList;
|
||||||
import org.apache.hadoop.util.StringUtils;
|
import org.apache.hadoop.util.StringUtils;
|
||||||
|
|
||||||
|
@ -46,8 +47,33 @@ import com.google.common.primitives.SignedBytes;
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public abstract class INode implements Comparable<byte[]> {
|
public abstract class INode implements Comparable<byte[]> {
|
||||||
|
/** A dummy INode which can be used as a probe object. */
|
||||||
|
public static final INode DUMMY = new INode() {
|
||||||
|
@Override
|
||||||
|
int collectSubtreeBlocksAndClear(BlocksMapUpdateInfo info) {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
@Override
|
||||||
|
long[] computeContentSummary(long[] summary) {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
@Override
|
||||||
|
DirCounts spaceConsumedInTree(DirCounts counts) {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
};
|
||||||
static final ReadOnlyList<INode> EMPTY_READ_ONLY_LIST
|
static final ReadOnlyList<INode> EMPTY_READ_ONLY_LIST
|
||||||
= ReadOnlyList.Util.emptyList();
|
= ReadOnlyList.Util.emptyList();
|
||||||
|
/**
|
||||||
|
* Assert that the snapshot parameter must be null since
|
||||||
|
* this class only take care current state.
|
||||||
|
* Subclasses should override the methods for handling the snapshot states.
|
||||||
|
*/
|
||||||
|
static void assertNull(Snapshot snapshot) {
|
||||||
|
if (snapshot != null) {
|
||||||
|
throw new AssertionError("snapshot is not null: " + snapshot);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/** Wrapper of two counters for namespace consumed and diskspace consumed. */
|
/** Wrapper of two counters for namespace consumed and diskspace consumed. */
|
||||||
|
@ -120,9 +146,12 @@ public abstract class INode implements Comparable<byte[]> {
|
||||||
* should not modify it.
|
* should not modify it.
|
||||||
*/
|
*/
|
||||||
private long permission = 0L;
|
private long permission = 0L;
|
||||||
protected INodeDirectory parent = null;
|
INodeDirectory parent = null;
|
||||||
protected long modificationTime = 0L;
|
private long modificationTime = 0L;
|
||||||
protected long accessTime = 0L;
|
private long accessTime = 0L;
|
||||||
|
|
||||||
|
/** For creating the a {@link #DUMMY} object. */
|
||||||
|
private INode() {}
|
||||||
|
|
||||||
private INode(byte[] name, long permission, INodeDirectory parent,
|
private INode(byte[] name, long permission, INodeDirectory parent,
|
||||||
long modificationTime, long accessTime) {
|
long modificationTime, long accessTime) {
|
||||||
|
@ -149,8 +178,8 @@ public abstract class INode implements Comparable<byte[]> {
|
||||||
|
|
||||||
/** @param other Other node to be copied */
|
/** @param other Other node to be copied */
|
||||||
INode(INode other) {
|
INode(INode other) {
|
||||||
this(other.getLocalNameBytes(), other.permission, other.getParent(),
|
this(other.name, other.permission, other.parent,
|
||||||
other.getModificationTime(), other.getAccessTime());
|
other.modificationTime, other.accessTime);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -290,13 +319,13 @@ public abstract class INode implements Comparable<byte[]> {
|
||||||
* Set local file name
|
* Set local file name
|
||||||
*/
|
*/
|
||||||
public void setLocalName(String name) {
|
public void setLocalName(String name) {
|
||||||
this.name = DFSUtil.string2Bytes(name);
|
setLocalName(DFSUtil.string2Bytes(name));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Set local file name
|
* Set local file name
|
||||||
*/
|
*/
|
||||||
void setLocalName(byte[] name) {
|
public void setLocalName(byte[] name) {
|
||||||
this.name = name;
|
this.name = name;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -316,7 +345,7 @@ public abstract class INode implements Comparable<byte[]> {
|
||||||
* Get parent directory
|
* Get parent directory
|
||||||
* @return parent INode
|
* @return parent INode
|
||||||
*/
|
*/
|
||||||
INodeDirectory getParent() {
|
public INodeDirectory getParent() {
|
||||||
return this.parent;
|
return this.parent;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -336,17 +365,21 @@ public abstract class INode implements Comparable<byte[]> {
|
||||||
/**
|
/**
|
||||||
* Set last modification time of inode.
|
* Set last modification time of inode.
|
||||||
*/
|
*/
|
||||||
public void setModificationTime(long modtime) {
|
public void updateModificationTime(long modtime) {
|
||||||
assert isDirectory();
|
assert isDirectory();
|
||||||
if (this.modificationTime <= modtime) {
|
if (this.modificationTime <= modtime) {
|
||||||
this.modificationTime = modtime;
|
setModificationTime(modtime);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void cloneModificationTime(INode that) {
|
||||||
|
this.modificationTime = that.modificationTime;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Always set the last modification time of inode.
|
* Always set the last modification time of inode.
|
||||||
*/
|
*/
|
||||||
void setModificationTimeForce(long modtime) {
|
void setModificationTime(long modtime) {
|
||||||
this.modificationTime = modtime;
|
this.modificationTime = modtime;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -431,11 +464,11 @@ public abstract class INode implements Comparable<byte[]> {
|
||||||
return buf.toString();
|
return buf.toString();
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean removeNode() {
|
public boolean removeNode(Snapshot latestSnapshot) {
|
||||||
if (parent == null) {
|
if (parent == null) {
|
||||||
return false;
|
return false;
|
||||||
} else {
|
} else {
|
||||||
parent.removeChild(this);
|
parent.removeChild(this, latestSnapshot);
|
||||||
parent = null;
|
parent = null;
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
|
@ -54,7 +54,7 @@ public class INodeDirectory extends INode {
|
||||||
}
|
}
|
||||||
return (INodeDirectory)inode;
|
return (INodeDirectory)inode;
|
||||||
}
|
}
|
||||||
|
|
||||||
protected static final int DEFAULT_FILES_PER_DIRECTORY = 5;
|
protected static final int DEFAULT_FILES_PER_DIRECTORY = 5;
|
||||||
final static String ROOT_NAME = "";
|
final static String ROOT_NAME = "";
|
||||||
|
|
||||||
|
@ -99,32 +99,56 @@ public class INodeDirectory extends INode {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private int searchChildren(INode inode) {
|
public int searchChildren(INode inode) {
|
||||||
return Collections.binarySearch(children, inode.getLocalNameBytes());
|
return Collections.binarySearch(children, inode.getLocalNameBytes());
|
||||||
}
|
}
|
||||||
|
|
||||||
INode removeChild(INode node) {
|
public int searchChildrenForExistingINode(INode inode) {
|
||||||
|
final int i = searchChildren(inode);
|
||||||
|
if (i < 0) {
|
||||||
|
throw new AssertionError("Child not found: inode=" + inode);
|
||||||
|
}
|
||||||
|
return i;
|
||||||
|
}
|
||||||
|
|
||||||
|
public INode removeChild(INode node, Snapshot latestSnapshot) {
|
||||||
assertChildrenNonNull();
|
assertChildrenNonNull();
|
||||||
|
|
||||||
|
if (latestSnapshot != null) {
|
||||||
|
final INodeDirectoryWithSnapshot dir
|
||||||
|
= INodeDirectoryWithSnapshot.replaceDir(this, latestSnapshot);
|
||||||
|
return dir.removeChild(node, latestSnapshot);
|
||||||
|
}
|
||||||
|
|
||||||
final int i = searchChildren(node);
|
final int i = searchChildren(node);
|
||||||
return i >= 0? children.remove(i): null;
|
return i >= 0? children.remove(i): null;
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Replace a child that has the same name as newChild by newChild.
|
/** Replace a child that has the same name as newChild by newChild.
|
||||||
*
|
*
|
||||||
* @param newChild Child node to be added
|
* @param newChild Child node to be added
|
||||||
*/
|
*/
|
||||||
void replaceChild(INode newChild) {
|
public INode replaceChild(INodeDirectory newChild, Snapshot latestSnapshot) {
|
||||||
assertChildrenNonNull();
|
assertChildrenNonNull();
|
||||||
|
|
||||||
final int low = searchChildren(newChild);
|
if (latestSnapshot != null) {
|
||||||
if (low>=0) { // an old child exists so replace by the newChild
|
final INodeDirectoryWithSnapshot dir
|
||||||
children.set(low, newChild);
|
= INodeDirectoryWithSnapshot.replaceDir(this, latestSnapshot);
|
||||||
} else {
|
return dir.replaceChild(newChild, latestSnapshot);
|
||||||
throw new IllegalArgumentException("No child exists to be replaced");
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// find the old child and replace it
|
||||||
|
final int low = searchChildrenForExistingINode(newChild);
|
||||||
|
final INode oldChild = children.set(low, newChild);
|
||||||
|
// set the parent of the children of the child.
|
||||||
|
for(INode i : newChild.getChildrenList(null)) {
|
||||||
|
i.parent = newChild;
|
||||||
|
}
|
||||||
|
return oldChild;
|
||||||
}
|
}
|
||||||
|
|
||||||
private INode getChild(byte[] name, Snapshot snapshot) {
|
public INode getChild(byte[] name, Snapshot snapshot) {
|
||||||
|
assertNull(snapshot);
|
||||||
|
|
||||||
final ReadOnlyList<INode> c = getChildrenList(snapshot);
|
final ReadOnlyList<INode> c = getChildrenList(snapshot);
|
||||||
final int i = ReadOnlyList.Util.binarySearch(c, name);
|
final int i = ReadOnlyList.Util.binarySearch(c, name);
|
||||||
return i < 0? null: c.get(i);
|
return i < 0? null: c.get(i);
|
||||||
|
@ -361,7 +385,14 @@ public class INodeDirectory extends INode {
|
||||||
* @return false if the child with this name already exists;
|
* @return false if the child with this name already exists;
|
||||||
* otherwise, return true;
|
* otherwise, return true;
|
||||||
*/
|
*/
|
||||||
public boolean addChild(final INode node, final boolean setModTime) {
|
public boolean addChild(final INode node, final boolean setModTime,
|
||||||
|
Snapshot latestSnapshot) {
|
||||||
|
if (latestSnapshot != null) {
|
||||||
|
final INodeDirectoryWithSnapshot dir
|
||||||
|
= INodeDirectoryWithSnapshot.replaceDir(this, latestSnapshot);
|
||||||
|
return dir.addChild(node, setModTime, latestSnapshot);
|
||||||
|
}
|
||||||
|
|
||||||
if (children == null) {
|
if (children == null) {
|
||||||
children = new ArrayList<INode>(DEFAULT_FILES_PER_DIRECTORY);
|
children = new ArrayList<INode>(DEFAULT_FILES_PER_DIRECTORY);
|
||||||
}
|
}
|
||||||
|
@ -372,8 +403,9 @@ public class INodeDirectory extends INode {
|
||||||
node.parent = this;
|
node.parent = this;
|
||||||
children.add(-low - 1, node);
|
children.add(-low - 1, node);
|
||||||
// update modification time of the parent directory
|
// update modification time of the parent directory
|
||||||
if (setModTime)
|
if (setModTime) {
|
||||||
setModificationTime(node.getModificationTime());
|
updateModificationTime(node.getModificationTime());
|
||||||
|
}
|
||||||
if (node.getGroupName() == null) {
|
if (node.getGroupName() == null) {
|
||||||
node.setGroup(getGroupName());
|
node.setGroup(getGroupName());
|
||||||
}
|
}
|
||||||
|
@ -400,20 +432,28 @@ public class INodeDirectory extends INode {
|
||||||
}
|
}
|
||||||
newNode.setLocalName(pathComponents[pathComponents.length - 1]);
|
newNode.setLocalName(pathComponents[pathComponents.length - 1]);
|
||||||
// insert into the parent children list
|
// insert into the parent children list
|
||||||
INodeDirectory parent = getParent(pathComponents);
|
INodesInPath inodes = getExistingPathINodes(pathComponents, 2, false);
|
||||||
return parent.addChild(newNode, true);
|
INodeDirectory parent = INodeDirectory.valueOf(inodes.inodes[0], pathComponents);
|
||||||
|
return parent.addChild(newNode, true, inodes.getLatestSnapshot());
|
||||||
}
|
}
|
||||||
|
|
||||||
INodeDirectory getParent(byte[][] pathComponents
|
INodeDirectory getParent(byte[][] pathComponents
|
||||||
) throws FileNotFoundException, PathIsNotDirectoryException,
|
) throws FileNotFoundException, PathIsNotDirectoryException,
|
||||||
UnresolvedLinkException {
|
UnresolvedLinkException {
|
||||||
|
return (INodeDirectory)getParentINodesInPath(pathComponents).getINode(0);
|
||||||
|
}
|
||||||
|
|
||||||
|
INodesInPath getParentINodesInPath(byte[][] pathComponents
|
||||||
|
) throws FileNotFoundException, PathIsNotDirectoryException,
|
||||||
|
UnresolvedLinkException {
|
||||||
if (pathComponents.length < 2) // add root
|
if (pathComponents.length < 2) // add root
|
||||||
return null;
|
return null;
|
||||||
// Gets the parent INode
|
// Gets the parent INode
|
||||||
INodesInPath inodes = getExistingPathINodes(pathComponents, 2, false);
|
INodesInPath inodes = getExistingPathINodes(pathComponents, 2, false);
|
||||||
return INodeDirectory.valueOf(inodes.inodes[0], pathComponents);
|
INodeDirectory.valueOf(inodes.inodes[0], pathComponents);
|
||||||
|
return inodes;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
DirCounts spaceConsumedInTree(DirCounts counts) {
|
DirCounts spaceConsumedInTree(DirCounts counts) {
|
||||||
counts.nsCount += 1;
|
counts.nsCount += 1;
|
||||||
|
@ -462,10 +502,11 @@ public class INodeDirectory extends INode {
|
||||||
* Note that the returned list is never null.
|
* Note that the returned list is never null.
|
||||||
*/
|
*/
|
||||||
public ReadOnlyList<INode> getChildrenList(final Snapshot snapshot) {
|
public ReadOnlyList<INode> getChildrenList(final Snapshot snapshot) {
|
||||||
//TODO: use snapshot to select children list
|
assertNull(snapshot);
|
||||||
return children == null ? EMPTY_READ_ONLY_LIST
|
return children == null ? EMPTY_READ_ONLY_LIST
|
||||||
: ReadOnlyList.Util.asReadOnlyList(children);
|
: ReadOnlyList.Util.asReadOnlyList(children);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Set the children list. */
|
/** Set the children list. */
|
||||||
public void setChildren(List<INode> children) {
|
public void setChildren(List<INode> children) {
|
||||||
this.children = children;
|
this.children = children;
|
||||||
|
@ -490,7 +531,7 @@ public class INodeDirectory extends INode {
|
||||||
* {@link INodeDirectory#getExistingPathINodes(byte[][], int, boolean)}.
|
* {@link INodeDirectory#getExistingPathINodes(byte[][], int, boolean)}.
|
||||||
* Contains INodes information resolved from a given path.
|
* Contains INodes information resolved from a given path.
|
||||||
*/
|
*/
|
||||||
static class INodesInPath {
|
public static class INodesInPath {
|
||||||
/**
|
/**
|
||||||
* Array with the specified number of INodes resolved for a given path.
|
* Array with the specified number of INodes resolved for a given path.
|
||||||
*/
|
*/
|
||||||
|
@ -570,7 +611,7 @@ public class INodeDirectory extends INode {
|
||||||
}
|
}
|
||||||
|
|
||||||
/** @return the i-th inode. */
|
/** @return the i-th inode. */
|
||||||
INode getINode(int i) {
|
public INode getINode(int i) {
|
||||||
return inodes[i];
|
return inodes[i];
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -676,4 +717,13 @@ public class INodeDirectory extends INode {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get last modification time of inode.
|
||||||
|
* @return access time
|
||||||
|
*/
|
||||||
|
public long getModificationTime(Snapshot snapshot) {
|
||||||
|
assertNull(snapshot);
|
||||||
|
return getModificationTime();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -89,16 +89,22 @@ public class INodeFile extends INode implements BlockCollection {
|
||||||
INodeFile(PermissionStatus permissions, BlockInfo[] blklist,
|
INodeFile(PermissionStatus permissions, BlockInfo[] blklist,
|
||||||
short replication, long modificationTime,
|
short replication, long modificationTime,
|
||||||
long atime, long preferredBlockSize) {
|
long atime, long preferredBlockSize) {
|
||||||
super(permissions, modificationTime, atime);
|
this(null, permissions, modificationTime, atime, blklist, replication,
|
||||||
|
preferredBlockSize);
|
||||||
|
}
|
||||||
|
|
||||||
|
INodeFile(byte[] name, PermissionStatus permissions, long mtime, long atime,
|
||||||
|
BlockInfo[] blklist, short replication, long preferredBlockSize) {
|
||||||
|
super(name, permissions, null, mtime, atime);
|
||||||
header = HeaderFormat.combineReplication(header, replication);
|
header = HeaderFormat.combineReplication(header, replication);
|
||||||
header = HeaderFormat.combinePreferredBlockSize(header, preferredBlockSize);
|
header = HeaderFormat.combinePreferredBlockSize(header, preferredBlockSize);
|
||||||
this.blocks = blklist;
|
this.blocks = blklist;
|
||||||
}
|
}
|
||||||
|
|
||||||
protected INodeFile(INodeFile f) {
|
protected INodeFile(INodeFile that) {
|
||||||
this(f.getPermissionStatus(), f.getBlocks(), f.getFileReplication(),
|
super(that);
|
||||||
f.getModificationTime(), f.getAccessTime(), f.getPreferredBlockSize());
|
this.header = that.header;
|
||||||
this.setLocalName(f.getLocalNameBytes());
|
this.blocks = that.blocks;
|
||||||
}
|
}
|
||||||
|
|
||||||
/** @return true unconditionally. */
|
/** @return true unconditionally. */
|
||||||
|
|
|
@ -72,9 +72,8 @@ public class INodeFileUnderConstruction extends INodeFile implements MutableBloc
|
||||||
String clientName,
|
String clientName,
|
||||||
String clientMachine,
|
String clientMachine,
|
||||||
DatanodeDescriptor clientNode) {
|
DatanodeDescriptor clientNode) {
|
||||||
super(perm, blocks, blockReplication, modificationTime, modificationTime,
|
super(name, perm, modificationTime, modificationTime,
|
||||||
preferredBlockSize);
|
blocks, blockReplication, preferredBlockSize);
|
||||||
setLocalName(name);
|
|
||||||
this.clientName = clientName;
|
this.clientName = clientName;
|
||||||
this.clientMachine = clientMachine;
|
this.clientMachine = clientMachine;
|
||||||
this.clientNode = clientNode;
|
this.clientNode = clientNode;
|
||||||
|
|
|
@ -331,22 +331,19 @@ public class LeaseManager {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
synchronized void changeLease(String src, String dst,
|
synchronized void changeLease(String src, String dst) {
|
||||||
String overwrite, String replaceBy) {
|
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug(getClass().getSimpleName() + ".changelease: " +
|
LOG.debug(getClass().getSimpleName() + ".changelease: " +
|
||||||
" src=" + src + ", dest=" + dst +
|
" src=" + src + ", dest=" + dst);
|
||||||
", overwrite=" + overwrite +
|
|
||||||
", replaceBy=" + replaceBy);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
final int len = overwrite.length();
|
final int len = src.length();
|
||||||
for(Map.Entry<String, Lease> entry
|
for(Map.Entry<String, Lease> entry
|
||||||
: findLeaseWithPrefixPath(src, sortedLeasesByPath).entrySet()) {
|
: findLeaseWithPrefixPath(src, sortedLeasesByPath).entrySet()) {
|
||||||
final String oldpath = entry.getKey();
|
final String oldpath = entry.getKey();
|
||||||
final Lease lease = entry.getValue();
|
final Lease lease = entry.getValue();
|
||||||
//overwrite must be a prefix of oldpath
|
// replace stem of src with new destination
|
||||||
final String newpath = replaceBy + oldpath.substring(len);
|
final String newpath = dst + oldpath.substring(len);
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug("changeLease: replacing " + oldpath + " with " + newpath);
|
LOG.debug("changeLease: replacing " + oldpath + " with " + newpath);
|
||||||
}
|
}
|
||||||
|
@ -429,6 +426,26 @@ public class LeaseManager {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the list of inodes corresponding to valid leases.
|
||||||
|
* @return list of inodes
|
||||||
|
* @throws UnresolvedLinkException
|
||||||
|
*/
|
||||||
|
Map<String, INodeFileUnderConstruction> getINodesUnderConstruction() {
|
||||||
|
Map<String, INodeFileUnderConstruction> inodes =
|
||||||
|
new TreeMap<String, INodeFileUnderConstruction>();
|
||||||
|
for (String p : sortedLeasesByPath.keySet()) {
|
||||||
|
// verify that path exists in namespace
|
||||||
|
try {
|
||||||
|
INode node = fsnamesystem.dir.getINode(p);
|
||||||
|
inodes.put(p, INodeFileUnderConstruction.valueOf(node, p));
|
||||||
|
} catch (IOException ioe) {
|
||||||
|
LOG.error(ioe);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return inodes;
|
||||||
|
}
|
||||||
|
|
||||||
/** Check the leases beginning from the oldest.
|
/** Check the leases beginning from the oldest.
|
||||||
* @return true is sync is needed.
|
* @return true is sync is needed.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -598,11 +598,7 @@ public class NameNode {
|
||||||
String nsId = getNameServiceId(conf);
|
String nsId = getNameServiceId(conf);
|
||||||
String namenodeId = HAUtil.getNameNodeId(conf, nsId);
|
String namenodeId = HAUtil.getNameNodeId(conf, nsId);
|
||||||
this.haEnabled = HAUtil.isHAEnabled(conf, nsId);
|
this.haEnabled = HAUtil.isHAEnabled(conf, nsId);
|
||||||
if (!haEnabled) {
|
state = createHAState();
|
||||||
state = ACTIVE_STATE;
|
|
||||||
} else {
|
|
||||||
state = STANDBY_STATE;
|
|
||||||
}
|
|
||||||
this.allowStaleStandbyReads = HAUtil.shouldAllowStandbyReads(conf);
|
this.allowStaleStandbyReads = HAUtil.shouldAllowStandbyReads(conf);
|
||||||
this.haContext = createHAContext();
|
this.haContext = createHAContext();
|
||||||
try {
|
try {
|
||||||
|
@ -619,6 +615,10 @@ public class NameNode {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected HAState createHAState() {
|
||||||
|
return !haEnabled ? ACTIVE_STATE : STANDBY_STATE;
|
||||||
|
}
|
||||||
|
|
||||||
protected HAContext createHAContext() {
|
protected HAContext createHAContext() {
|
||||||
return new NameNodeHAContext();
|
return new NameNodeHAContext();
|
||||||
}
|
}
|
||||||
|
@ -1298,7 +1298,7 @@ public class NameNode {
|
||||||
* before exit.
|
* before exit.
|
||||||
* @throws ExitException thrown only for testing.
|
* @throws ExitException thrown only for testing.
|
||||||
*/
|
*/
|
||||||
private synchronized void doImmediateShutdown(Throwable t)
|
protected synchronized void doImmediateShutdown(Throwable t)
|
||||||
throws ExitException {
|
throws ExitException {
|
||||||
String message = "Error encountered requiring NN shutdown. " +
|
String message = "Error encountered requiring NN shutdown. " +
|
||||||
"Shutting down immediately.";
|
"Shutting down immediately.";
|
||||||
|
|
|
@ -102,7 +102,7 @@ class NamenodeJspHelper {
|
||||||
long usedNonHeap = (totalNonHeap * 100) / commitedNonHeap;
|
long usedNonHeap = (totalNonHeap * 100) / commitedNonHeap;
|
||||||
|
|
||||||
String str = "<div>" + inodes + " files and directories, " + blocks + " blocks = "
|
String str = "<div>" + inodes + " files and directories, " + blocks + " blocks = "
|
||||||
+ (inodes + blocks) + " total";
|
+ (inodes + blocks) + " total filesystem objects";
|
||||||
if (maxobjects != 0) {
|
if (maxobjects != 0) {
|
||||||
long pct = ((inodes + blocks) * 100) / maxobjects;
|
long pct = ((inodes + blocks) * 100) / maxobjects;
|
||||||
str += " / " + maxobjects + " (" + pct + "%)";
|
str += " / " + maxobjects + " (" + pct + "%)";
|
||||||
|
|
|
@ -65,6 +65,19 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithQuota {
|
||||||
}
|
}
|
||||||
return (INodeDirectorySnapshottable)dir;
|
return (INodeDirectorySnapshottable)dir;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static Snapshot findLatestSnapshot(INode inode) {
|
||||||
|
Snapshot latest = null;
|
||||||
|
for(; inode != null; inode = inode.getParent()) {
|
||||||
|
if (inode instanceof INodeDirectorySnapshottable) {
|
||||||
|
final Snapshot s = ((INodeDirectorySnapshottable)inode).getLastSnapshot();
|
||||||
|
if (Snapshot.ID_COMPARATOR.compare(latest, s) < 0) {
|
||||||
|
latest = s;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return latest;
|
||||||
|
}
|
||||||
|
|
||||||
/** Snapshots of this directory in ascending order of snapshot id. */
|
/** Snapshots of this directory in ascending order of snapshot id. */
|
||||||
private final List<Snapshot> snapshots = new ArrayList<Snapshot>();
|
private final List<Snapshot> snapshots = new ArrayList<Snapshot>();
|
||||||
|
@ -196,8 +209,8 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithQuota {
|
||||||
|
|
||||||
//set modification time
|
//set modification time
|
||||||
final long timestamp = Time.now();
|
final long timestamp = Time.now();
|
||||||
s.getRoot().setModificationTime(timestamp);
|
s.getRoot().updateModificationTime(timestamp);
|
||||||
setModificationTime(timestamp);
|
updateModificationTime(timestamp);
|
||||||
return s;
|
return s;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -19,10 +19,15 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
|
import org.apache.hadoop.HadoopIllegalArgumentException;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.INode;
|
import org.apache.hadoop.hdfs.server.namenode.INode;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
|
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
|
||||||
|
import org.apache.hadoop.hdfs.util.ReadOnlyList;
|
||||||
|
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
|
||||||
/** The directory with snapshots. */
|
/** The directory with snapshots. */
|
||||||
public class INodeDirectoryWithSnapshot extends INodeDirectory {
|
public class INodeDirectoryWithSnapshot extends INodeDirectory {
|
||||||
|
@ -250,9 +255,223 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
|
||||||
+ "\n deleted=" + toString(deleted);
|
+ "\n deleted=" + toString(deleted);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private class SnapshotDiff implements Comparable<Snapshot> {
|
||||||
|
/** The snapshot will obtain after applied this diff. */
|
||||||
|
final Snapshot snapshot;
|
||||||
|
/** The size of the children list which is never changed. */
|
||||||
|
final int size;
|
||||||
|
/**
|
||||||
|
* Posterior diff is the diff happened after this diff.
|
||||||
|
* The posterior diff should be first applied to obtain the posterior
|
||||||
|
* snapshot and then apply this diff in order to obtain this snapshot.
|
||||||
|
* If the posterior diff is null, the posterior state is the current state.
|
||||||
|
*/
|
||||||
|
private SnapshotDiff posteriorDiff;
|
||||||
|
/** The data of this diff. */
|
||||||
|
private final Diff diff = new Diff();
|
||||||
|
/** The snapshot version of the inode. */
|
||||||
|
private INode snapshotINode;
|
||||||
|
|
||||||
INodeDirectoryWithSnapshot(String name, INodeDirectory dir) {
|
SnapshotDiff(Snapshot snapshot, int size) {
|
||||||
super(name, dir.getPermissionStatus());
|
if (size < 0) {
|
||||||
parent = dir;
|
throw new HadoopIllegalArgumentException("size = " + size + " < 0");
|
||||||
|
}
|
||||||
|
this.snapshot = snapshot;
|
||||||
|
this.size = size;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int compareTo(final Snapshot that_snapshot) {
|
||||||
|
return Snapshot.ID_COMPARATOR.compare(this.snapshot, that_snapshot);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The children list of a directory in a snapshot.
|
||||||
|
* Since the snapshot is read-only, the logical view of the list is
|
||||||
|
* never changed although the internal data structure may mutate.
|
||||||
|
*/
|
||||||
|
ReadOnlyList<INode> getChildrenList() {
|
||||||
|
return new ReadOnlyList<INode>() {
|
||||||
|
private List<INode> children = null;
|
||||||
|
|
||||||
|
private List<INode> initChildren() {
|
||||||
|
if (children == null) {
|
||||||
|
final ReadOnlyList<INode> posterior = posteriorDiff != null?
|
||||||
|
posteriorDiff.getChildrenList()
|
||||||
|
: INodeDirectoryWithSnapshot.this.getChildrenList(null);
|
||||||
|
children = diff.apply2Current(ReadOnlyList.Util.asList(posterior));
|
||||||
|
}
|
||||||
|
return children;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Iterator<INode> iterator() {
|
||||||
|
return initChildren().iterator();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isEmpty() {
|
||||||
|
return size == 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int size() {
|
||||||
|
return size;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public INode get(int i) {
|
||||||
|
return initChildren().get(i);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
INode getChild(byte[] name) {
|
||||||
|
final INode i = diff.accessPrevious(name, INode.DUMMY);
|
||||||
|
if (i != INode.DUMMY) {
|
||||||
|
// this diff is able to find it
|
||||||
|
return i;
|
||||||
|
} else {
|
||||||
|
// should return the posterior INode.
|
||||||
|
return posteriorDiff != null? posteriorDiff.getChild(name)
|
||||||
|
: INodeDirectoryWithSnapshot.this.getChild(name, null);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Replace the given directory to an {@link INodeDirectoryWithSnapshot}. */
|
||||||
|
public static INodeDirectoryWithSnapshot replaceDir(INodeDirectory oldDir,
|
||||||
|
Snapshot latestSnapshot) {
|
||||||
|
Preconditions.checkArgument(!(oldDir instanceof INodeDirectoryWithSnapshot),
|
||||||
|
"oldDir is already an INodeDirectoryWithSnapshot, oldDir=%s", oldDir);
|
||||||
|
|
||||||
|
final INodeDirectory parent = oldDir.getParent();
|
||||||
|
Preconditions.checkArgument(parent != null,
|
||||||
|
"parent is null, oldDir=%s", oldDir);
|
||||||
|
|
||||||
|
final INodeDirectoryWithSnapshot newDir = new INodeDirectoryWithSnapshot(
|
||||||
|
oldDir, latestSnapshot);
|
||||||
|
parent.replaceChild(newDir, null);
|
||||||
|
return newDir;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Diff list sorted by snapshot IDs, i.e. in chronological order. */
|
||||||
|
private final List<SnapshotDiff> diffs = new ArrayList<SnapshotDiff>();
|
||||||
|
|
||||||
|
INodeDirectoryWithSnapshot(INodeDirectory that, Snapshot s) {
|
||||||
|
super(that);
|
||||||
|
|
||||||
|
// add a diff for the snapshot
|
||||||
|
addSnapshotDiff(s, that.getChildrenList(null).size());
|
||||||
|
}
|
||||||
|
|
||||||
|
INodeDirectoryWithSnapshot(String name, INodeDirectory dir, Snapshot s) {
|
||||||
|
this(dir, s);
|
||||||
|
setLocalName(name);
|
||||||
|
setParent(dir);
|
||||||
|
}
|
||||||
|
|
||||||
|
SnapshotDiff addSnapshotDiff(Snapshot snapshot, int childrenSize) {
|
||||||
|
final SnapshotDiff d = new SnapshotDiff(snapshot, childrenSize);
|
||||||
|
diffs.add(d);
|
||||||
|
return d;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check if the latest snapshot diff exist. If not, add it.
|
||||||
|
* @return the latest snapshot diff, which is never null.
|
||||||
|
*/
|
||||||
|
private SnapshotDiff checkAndAddLatestSnapshotDiff(Snapshot latest) {
|
||||||
|
final SnapshotDiff last = getLastSnapshotDiff();
|
||||||
|
if (last != null && last.snapshot.equals(latest)) {
|
||||||
|
return last;
|
||||||
|
}
|
||||||
|
|
||||||
|
final int size = getChildrenList(null).size();
|
||||||
|
final SnapshotDiff d = addSnapshotDiff(latest, size);
|
||||||
|
if (last != null) {
|
||||||
|
last.posteriorDiff = d;
|
||||||
|
}
|
||||||
|
return d;
|
||||||
|
}
|
||||||
|
|
||||||
|
Diff getLatestDiff(Snapshot latest) {
|
||||||
|
return checkAndAddLatestSnapshotDiff(latest).diff;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the diff corresponding to the snapshot.
|
||||||
|
* When the diff is not found, it means that the current state and
|
||||||
|
* the snapshot state are the same.
|
||||||
|
*/
|
||||||
|
SnapshotDiff getSnapshotDiff(Snapshot snapshot) {
|
||||||
|
if (snapshot == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
final int i = Collections.binarySearch(diffs, snapshot);
|
||||||
|
if (i >= 0) {
|
||||||
|
// exact match
|
||||||
|
return diffs.get(i);
|
||||||
|
} else {
|
||||||
|
// Exact match not found means that there were no changes between
|
||||||
|
// given snapshot and the next state so that the diff for the given
|
||||||
|
// snapshot is not recorded. Thus, use the next state.
|
||||||
|
final int j = -i - 1;
|
||||||
|
return j < diffs.size()? diffs.get(j): null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
SnapshotDiff getLastSnapshotDiff() {
|
||||||
|
return diffs.get(diffs.size() - 1);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ReadOnlyList<INode> getChildrenList(Snapshot snapshot) {
|
||||||
|
final SnapshotDiff diff = getSnapshotDiff(snapshot);
|
||||||
|
if (diff != null) {
|
||||||
|
return diff.getChildrenList();
|
||||||
|
}
|
||||||
|
return super.getChildrenList(null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public INode getChild(byte[] name, Snapshot snapshot) {
|
||||||
|
final SnapshotDiff diff = getSnapshotDiff(snapshot);
|
||||||
|
if (diff != null) {
|
||||||
|
return diff.getChild(name);
|
||||||
|
}
|
||||||
|
return super.getChild(name, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean addChild(INode inode, boolean setModTime,
|
||||||
|
Snapshot latestSnapshot) {
|
||||||
|
getLatestDiff(latestSnapshot).create(inode);
|
||||||
|
return super.addChild(inode, setModTime, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public INode removeChild(INode inode, Snapshot latestSnapshot) {
|
||||||
|
getLatestDiff(latestSnapshot).delete(inode);
|
||||||
|
return super.removeChild(inode, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public INode replaceChild(INodeDirectory newChild, Snapshot latestSnapshot) {
|
||||||
|
final INode oldChild = super.replaceChild(newChild, null);
|
||||||
|
final Diff diff = getLatestDiff(latestSnapshot);
|
||||||
|
diff.delete(oldChild);
|
||||||
|
diff.create(newChild);
|
||||||
|
return oldChild;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getModificationTime(Snapshot snapshot) {
|
||||||
|
final SnapshotDiff diff = getSnapshotDiff(snapshot);
|
||||||
|
if (diff != null) {
|
||||||
|
return diff.snapshotINode.getModificationTime();
|
||||||
|
}
|
||||||
|
return getModificationTime();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -109,7 +109,7 @@ public class INodeFileWithLink extends INodeFile {
|
||||||
this.setFileReplication(maxReplication);
|
this.setFileReplication(maxReplication);
|
||||||
this.next = null;
|
this.next = null;
|
||||||
// clear parent
|
// clear parent
|
||||||
parent = null;
|
setParent(null);
|
||||||
}
|
}
|
||||||
return 1;
|
return 1;
|
||||||
}
|
}
|
||||||
|
|
|
@ -44,7 +44,7 @@ public class Snapshot implements Comparable<byte[]> {
|
||||||
|
|
||||||
Snapshot(int id, String name, INodeDirectorySnapshottable dir) {
|
Snapshot(int id, String name, INodeDirectorySnapshottable dir) {
|
||||||
this.id = id;
|
this.id = id;
|
||||||
this.root = new INodeDirectoryWithSnapshot(name, dir);
|
this.root = new INodeDirectoryWithSnapshot(name, dir, this);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** @return the root directory of the snapshot. */
|
/** @return the root directory of the snapshot. */
|
||||||
|
@ -57,6 +57,21 @@ public class Snapshot implements Comparable<byte[]> {
|
||||||
return root.compareTo(bytes);
|
return root.compareTo(bytes);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object that) {
|
||||||
|
if (this == that) {
|
||||||
|
return true;
|
||||||
|
} else if (that == null || !(that instanceof Snapshot)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
return this.id == ((Snapshot)that).id;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return id;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return getClass().getSimpleName() + ":" + root.getLocalName();
|
return getClass().getSimpleName() + ":" + root.getLocalName();
|
||||||
|
|
|
@ -24,12 +24,8 @@ import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
|
import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.INode;
|
|
||||||
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
|
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
|
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory.INodesInPath;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.INodeFileUnderConstruction;
|
|
||||||
import org.apache.hadoop.hdfs.server.namenode.INodeSymlink;
|
|
||||||
import org.apache.hadoop.hdfs.util.ReadOnlyList;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Manage snapshottable directories and their snapshots.
|
* Manage snapshottable directories and their snapshots.
|
||||||
|
@ -68,7 +64,9 @@ public class SnapshotManager implements SnapshotStats {
|
||||||
*/
|
*/
|
||||||
public void setSnapshottable(final String path, final int snapshotQuota
|
public void setSnapshottable(final String path, final int snapshotQuota
|
||||||
) throws IOException {
|
) throws IOException {
|
||||||
final INodeDirectory d = INodeDirectory.valueOf(fsdir.getINode(path), path);
|
final INodesInPath inodesInPath = fsdir.getINodesInPath(path);
|
||||||
|
final INodeDirectory d = INodeDirectory.valueOf(
|
||||||
|
inodesInPath.getINode(0), path);
|
||||||
if (d.isSnapshottable()) {
|
if (d.isSnapshottable()) {
|
||||||
//The directory is already a snapshottable directory.
|
//The directory is already a snapshottable directory.
|
||||||
((INodeDirectorySnapshottable)d).setSnapshotQuota(snapshotQuota);
|
((INodeDirectorySnapshottable)d).setSnapshotQuota(snapshotQuota);
|
||||||
|
@ -77,7 +75,7 @@ public class SnapshotManager implements SnapshotStats {
|
||||||
|
|
||||||
final INodeDirectorySnapshottable s
|
final INodeDirectorySnapshottable s
|
||||||
= INodeDirectorySnapshottable.newInstance(d, snapshotQuota);
|
= INodeDirectorySnapshottable.newInstance(d, snapshotQuota);
|
||||||
fsdir.replaceINodeDirectory(path, d, s);
|
fsdir.replaceINodeDirectory(path, d, s, inodesInPath.getLatestSnapshot());
|
||||||
snapshottables.add(s);
|
snapshottables.add(s);
|
||||||
|
|
||||||
numSnapshottableDirs.getAndIncrement();
|
numSnapshottableDirs.getAndIncrement();
|
||||||
|
@ -90,15 +88,16 @@ public class SnapshotManager implements SnapshotStats {
|
||||||
*/
|
*/
|
||||||
public void resetSnapshottable(final String path
|
public void resetSnapshottable(final String path
|
||||||
) throws IOException {
|
) throws IOException {
|
||||||
|
final INodesInPath inodesInPath = fsdir.getINodesInPath(path);
|
||||||
final INodeDirectorySnapshottable s = INodeDirectorySnapshottable.valueOf(
|
final INodeDirectorySnapshottable s = INodeDirectorySnapshottable.valueOf(
|
||||||
fsdir.getINode(path), path);
|
inodesInPath.getINode(0), path);
|
||||||
if (s.getNumSnapshots() > 0) {
|
if (s.getNumSnapshots() > 0) {
|
||||||
throw new SnapshotException("The directory " + path + " has snapshot(s). "
|
throw new SnapshotException("The directory " + path + " has snapshot(s). "
|
||||||
+ "Please redo the operation after removing all the snapshots.");
|
+ "Please redo the operation after removing all the snapshots.");
|
||||||
}
|
}
|
||||||
|
|
||||||
final INodeDirectory d = new INodeDirectory(s);
|
final INodeDirectory d = new INodeDirectory(s);
|
||||||
fsdir.replaceINodeDirectory(path, s, d);
|
fsdir.replaceINodeDirectory(path, s, d, inodesInPath.getLatestSnapshot());
|
||||||
snapshottables.remove(s);
|
snapshottables.remove(s);
|
||||||
|
|
||||||
numSnapshottableDirs.getAndDecrement();
|
numSnapshottableDirs.getAndDecrement();
|
||||||
|
@ -121,9 +120,8 @@ public class SnapshotManager implements SnapshotStats {
|
||||||
// Find the source root directory path where the snapshot is taken.
|
// Find the source root directory path where the snapshot is taken.
|
||||||
final INodeDirectorySnapshottable srcRoot
|
final INodeDirectorySnapshottable srcRoot
|
||||||
= INodeDirectorySnapshottable.valueOf(fsdir.getINode(path), path);
|
= INodeDirectorySnapshottable.valueOf(fsdir.getINode(path), path);
|
||||||
final Snapshot s = srcRoot.addSnapshot(snapshotID, snapshotName);
|
srcRoot.addSnapshot(snapshotID, snapshotName);
|
||||||
new SnapshotCreation().processRecursively(srcRoot, s.getRoot());
|
|
||||||
|
|
||||||
//create success, update id
|
//create success, update id
|
||||||
snapshotID++;
|
snapshotID++;
|
||||||
numSnapshots.getAndIncrement();
|
numSnapshots.getAndIncrement();
|
||||||
|
@ -154,85 +152,6 @@ public class SnapshotManager implements SnapshotStats {
|
||||||
srcRoot.renameSnapshot(path, oldSnapshotName, newSnapshotName);
|
srcRoot.renameSnapshot(path, oldSnapshotName, newSnapshotName);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Create a snapshot of subtrees by recursively coping the directory
|
|
||||||
* structure from the source directory to the snapshot destination directory.
|
|
||||||
* This creation algorithm requires O(N) running time and O(N) memory,
|
|
||||||
* where N = # files + # directories + # symlinks.
|
|
||||||
*/
|
|
||||||
class SnapshotCreation {
|
|
||||||
/** Process snapshot creation recursively. */
|
|
||||||
private void processRecursively(final INodeDirectory srcDir,
|
|
||||||
final INodeDirectory dstDir) throws IOException {
|
|
||||||
final ReadOnlyList<INode> children = srcDir.getChildrenList(null);
|
|
||||||
if (!children.isEmpty()) {
|
|
||||||
final List<INode> inodes = new ArrayList<INode>(children.size());
|
|
||||||
for(final INode c : new ArrayList<INode>(ReadOnlyList.Util.asList(children))) {
|
|
||||||
final INode i;
|
|
||||||
if (c == null) {
|
|
||||||
i = null;
|
|
||||||
} else if (c instanceof INodeDirectory) {
|
|
||||||
//also handle INodeDirectoryWithQuota
|
|
||||||
i = processINodeDirectory((INodeDirectory)c);
|
|
||||||
} else if (c instanceof INodeFileUnderConstruction) {
|
|
||||||
//TODO: support INodeFileUnderConstruction
|
|
||||||
throw new IOException("Not yet supported.");
|
|
||||||
} else if (c instanceof INodeFile) {
|
|
||||||
i = processINodeFile(srcDir, (INodeFile)c);
|
|
||||||
} else if (c instanceof INodeSymlink) {
|
|
||||||
i = new INodeSymlink((INodeSymlink)c);
|
|
||||||
} else {
|
|
||||||
throw new AssertionError("Unknow INode type: " + c.getClass()
|
|
||||||
+ ", inode = " + c);
|
|
||||||
}
|
|
||||||
i.setParent(dstDir);
|
|
||||||
inodes.add(i);
|
|
||||||
}
|
|
||||||
dstDir.setChildren(inodes);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Create destination INodeDirectory and make the recursive call.
|
|
||||||
* @return destination INodeDirectory.
|
|
||||||
*/
|
|
||||||
private INodeDirectory processINodeDirectory(final INodeDirectory srcChild
|
|
||||||
) throws IOException {
|
|
||||||
final INodeDirectory dstChild = new INodeDirectory(srcChild);
|
|
||||||
dstChild.setChildren(null);
|
|
||||||
processRecursively(srcChild, dstChild);
|
|
||||||
return dstChild;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Create destination INodeFileSnapshot and update source INode type.
|
|
||||||
* @return destination INodeFileSnapshot.
|
|
||||||
*/
|
|
||||||
private INodeFileSnapshot processINodeFile(final INodeDirectory parent,
|
|
||||||
final INodeFile file) {
|
|
||||||
final INodeFileSnapshot snapshot = new INodeFileSnapshot(
|
|
||||||
file, file.computeFileSize(true));
|
|
||||||
|
|
||||||
final INodeFileWithLink srcWithLink;
|
|
||||||
//check source INode type
|
|
||||||
if (file instanceof INodeFileWithLink) {
|
|
||||||
srcWithLink = (INodeFileWithLink)file;
|
|
||||||
} else {
|
|
||||||
//source is an INodeFile, replace the source.
|
|
||||||
srcWithLink = new INodeFileWithLink(file);
|
|
||||||
file.removeNode();
|
|
||||||
parent.addChild(srcWithLink, false);
|
|
||||||
|
|
||||||
//update block map
|
|
||||||
namesystem.getBlockManager().addBlockCollection(srcWithLink);
|
|
||||||
}
|
|
||||||
|
|
||||||
//insert the snapshot to src's linked list.
|
|
||||||
srcWithLink.insert(snapshot);
|
|
||||||
return snapshot;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getNumSnapshottableDirs() {
|
public long getNumSnapshottableDirs() {
|
||||||
return numSnapshottableDirs.get();
|
return numSnapshottableDirs.get();
|
||||||
|
@ -243,4 +162,4 @@ public class SnapshotManager implements SnapshotStats {
|
||||||
return numSnapshots.get();
|
return numSnapshots.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -30,7 +30,9 @@ import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Options;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
||||||
|
@ -49,6 +51,10 @@ public class TestLease {
|
||||||
).getLeaseByPath(src.toString()) != null;
|
).getLeaseByPath(src.toString()) != null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
static int leaseCount(MiniDFSCluster cluster) {
|
||||||
|
return NameNodeAdapter.getLeaseManager(cluster.getNamesystem()).countLease();
|
||||||
|
}
|
||||||
|
|
||||||
static final String dirString = "/test/lease";
|
static final String dirString = "/test/lease";
|
||||||
final Path dir = new Path(dirString);
|
final Path dir = new Path(dirString);
|
||||||
static final Log LOG = LogFactory.getLog(TestLease.class);
|
static final Log LOG = LogFactory.getLog(TestLease.class);
|
||||||
|
@ -126,6 +132,96 @@ public class TestLease {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testLeaseAfterRename() throws Exception {
|
||||||
|
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
|
||||||
|
try {
|
||||||
|
Path p = new Path("/test-file");
|
||||||
|
Path d = new Path("/test-d");
|
||||||
|
Path d2 = new Path("/test-d-other");
|
||||||
|
|
||||||
|
// open a file to get a lease
|
||||||
|
FileSystem fs = cluster.getFileSystem();
|
||||||
|
FSDataOutputStream out = fs.create(p);
|
||||||
|
out.writeBytes("something");
|
||||||
|
//out.hsync();
|
||||||
|
Assert.assertTrue(hasLease(cluster, p));
|
||||||
|
Assert.assertEquals(1, leaseCount(cluster));
|
||||||
|
|
||||||
|
// just to ensure first fs doesn't have any logic to twiddle leases
|
||||||
|
DistributedFileSystem fs2 = (DistributedFileSystem) FileSystem.newInstance(fs.getUri(), fs.getConf());
|
||||||
|
|
||||||
|
// rename the file into an existing dir
|
||||||
|
LOG.info("DMS: rename file into dir");
|
||||||
|
Path pRenamed = new Path(d, p.getName());
|
||||||
|
fs2.mkdirs(d);
|
||||||
|
fs2.rename(p, pRenamed);
|
||||||
|
Assert.assertFalse(p+" exists", fs2.exists(p));
|
||||||
|
Assert.assertTrue(pRenamed+" not found", fs2.exists(pRenamed));
|
||||||
|
Assert.assertFalse("has lease for "+p, hasLease(cluster, p));
|
||||||
|
Assert.assertTrue("no lease for "+pRenamed, hasLease(cluster, pRenamed));
|
||||||
|
Assert.assertEquals(1, leaseCount(cluster));
|
||||||
|
|
||||||
|
// rename the parent dir to a new non-existent dir
|
||||||
|
LOG.info("DMS: rename parent dir");
|
||||||
|
Path pRenamedAgain = new Path(d2, pRenamed.getName());
|
||||||
|
fs2.rename(d, d2);
|
||||||
|
// src gone
|
||||||
|
Assert.assertFalse(d+" exists", fs2.exists(d));
|
||||||
|
Assert.assertFalse("has lease for "+pRenamed, hasLease(cluster, pRenamed));
|
||||||
|
// dst checks
|
||||||
|
Assert.assertTrue(d2+" not found", fs2.exists(d2));
|
||||||
|
Assert.assertTrue(pRenamedAgain+" not found", fs2.exists(pRenamedAgain));
|
||||||
|
Assert.assertTrue("no lease for "+pRenamedAgain, hasLease(cluster, pRenamedAgain));
|
||||||
|
Assert.assertEquals(1, leaseCount(cluster));
|
||||||
|
|
||||||
|
// rename the parent dir to existing dir
|
||||||
|
// NOTE: rename w/o options moves paths into existing dir
|
||||||
|
LOG.info("DMS: rename parent again");
|
||||||
|
pRenamed = pRenamedAgain;
|
||||||
|
pRenamedAgain = new Path(new Path(d, d2.getName()), p.getName());
|
||||||
|
fs2.mkdirs(d);
|
||||||
|
fs2.rename(d2, d);
|
||||||
|
// src gone
|
||||||
|
Assert.assertFalse(d2+" exists", fs2.exists(d2));
|
||||||
|
Assert.assertFalse("no lease for "+pRenamed, hasLease(cluster, pRenamed));
|
||||||
|
// dst checks
|
||||||
|
Assert.assertTrue(d+" not found", fs2.exists(d));
|
||||||
|
Assert.assertTrue(pRenamedAgain +" not found", fs2.exists(pRenamedAgain));
|
||||||
|
Assert.assertTrue("no lease for "+pRenamedAgain, hasLease(cluster, pRenamedAgain));
|
||||||
|
Assert.assertEquals(1, leaseCount(cluster));
|
||||||
|
|
||||||
|
// rename with opts to non-existent dir
|
||||||
|
pRenamed = pRenamedAgain;
|
||||||
|
pRenamedAgain = new Path(d2, p.getName());
|
||||||
|
fs2.rename(pRenamed.getParent(), d2, Options.Rename.OVERWRITE);
|
||||||
|
// src gone
|
||||||
|
Assert.assertFalse(pRenamed.getParent() +" not found", fs2.exists(pRenamed.getParent()));
|
||||||
|
Assert.assertFalse("has lease for "+pRenamed, hasLease(cluster, pRenamed));
|
||||||
|
// dst checks
|
||||||
|
Assert.assertTrue(d2+" not found", fs2.exists(d2));
|
||||||
|
Assert.assertTrue(pRenamedAgain+" not found", fs2.exists(pRenamedAgain));
|
||||||
|
Assert.assertTrue("no lease for "+pRenamedAgain, hasLease(cluster, pRenamedAgain));
|
||||||
|
Assert.assertEquals(1, leaseCount(cluster));
|
||||||
|
|
||||||
|
// rename with opts to existing dir
|
||||||
|
// NOTE: rename with options will not move paths into the existing dir
|
||||||
|
pRenamed = pRenamedAgain;
|
||||||
|
pRenamedAgain = new Path(d, p.getName());
|
||||||
|
fs2.rename(pRenamed.getParent(), d, Options.Rename.OVERWRITE);
|
||||||
|
// src gone
|
||||||
|
Assert.assertFalse(pRenamed.getParent() +" not found", fs2.exists(pRenamed.getParent()));
|
||||||
|
Assert.assertFalse("has lease for "+pRenamed, hasLease(cluster, pRenamed));
|
||||||
|
// dst checks
|
||||||
|
Assert.assertTrue(d+" not found", fs2.exists(d));
|
||||||
|
Assert.assertTrue(pRenamedAgain+" not found", fs2.exists(pRenamedAgain));
|
||||||
|
Assert.assertTrue("no lease for "+pRenamedAgain, hasLease(cluster, pRenamedAgain));
|
||||||
|
Assert.assertEquals(1, leaseCount(cluster));
|
||||||
|
} finally {
|
||||||
|
cluster.shutdown();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testLease() throws Exception {
|
public void testLease() throws Exception {
|
||||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
|
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
|
||||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.FileUtil;
|
import org.apache.hadoop.fs.FileUtil;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
|
||||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||||
import org.apache.hadoop.hdfs.HAUtil;
|
import org.apache.hadoop.hdfs.HAUtil;
|
||||||
|
@ -103,6 +104,9 @@ public class TestBackupNode {
|
||||||
BackupNode bn = (BackupNode)NameNode.createNameNode(
|
BackupNode bn = (BackupNode)NameNode.createNameNode(
|
||||||
new String[]{startupOpt.getName()}, c);
|
new String[]{startupOpt.getName()}, c);
|
||||||
assertTrue(bn.getRole() + " must be in SafeMode.", bn.isInSafeMode());
|
assertTrue(bn.getRole() + " must be in SafeMode.", bn.isInSafeMode());
|
||||||
|
assertTrue(bn.getRole() + " must be in StandbyState",
|
||||||
|
bn.getNamesystem().getHAState()
|
||||||
|
.equalsIgnoreCase(HAServiceState.STANDBY.name()));
|
||||||
return bn;
|
return bn;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -158,7 +158,7 @@ public class TestFsLimits {
|
||||||
Class<?> generated = null;
|
Class<?> generated = null;
|
||||||
try {
|
try {
|
||||||
fs.verifyFsLimits(inodes, 1, child);
|
fs.verifyFsLimits(inodes, 1, child);
|
||||||
rootInode.addChild(child, false);
|
rootInode.addChild(child, false, null);
|
||||||
} catch (QuotaExceededException e) {
|
} catch (QuotaExceededException e) {
|
||||||
generated = e.getClass();
|
generated = e.getClass();
|
||||||
}
|
}
|
||||||
|
|
|
@ -144,11 +144,11 @@ public class TestINodeFile {
|
||||||
assertEquals("f", inf.getFullPathName());
|
assertEquals("f", inf.getFullPathName());
|
||||||
assertEquals("", inf.getLocalParentDir());
|
assertEquals("", inf.getLocalParentDir());
|
||||||
|
|
||||||
dir.addChild(inf, false);
|
dir.addChild(inf, false, null);
|
||||||
assertEquals("d"+Path.SEPARATOR+"f", inf.getFullPathName());
|
assertEquals("d"+Path.SEPARATOR+"f", inf.getFullPathName());
|
||||||
assertEquals("d", inf.getLocalParentDir());
|
assertEquals("d", inf.getLocalParentDir());
|
||||||
|
|
||||||
root.addChild(dir, false);
|
root.addChild(dir, false, null);
|
||||||
assertEquals(Path.SEPARATOR+"d"+Path.SEPARATOR+"f", inf.getFullPathName());
|
assertEquals(Path.SEPARATOR+"d"+Path.SEPARATOR+"f", inf.getFullPathName());
|
||||||
assertEquals(Path.SEPARATOR+"d", dir.getFullPathName());
|
assertEquals(Path.SEPARATOR+"d", dir.getFullPathName());
|
||||||
|
|
||||||
|
|
|
@ -92,7 +92,8 @@ public class TestMetaSave {
|
||||||
DataInputStream in = new DataInputStream(fstream);
|
DataInputStream in = new DataInputStream(fstream);
|
||||||
BufferedReader reader = new BufferedReader(new InputStreamReader(in));
|
BufferedReader reader = new BufferedReader(new InputStreamReader(in));
|
||||||
String line = reader.readLine();
|
String line = reader.readLine();
|
||||||
assertTrue(line.equals("3 files and directories, 2 blocks = 5 total"));
|
assertTrue(line.equals(
|
||||||
|
"3 files and directories, 2 blocks = 5 total filesystem objects"));
|
||||||
line = reader.readLine();
|
line = reader.readLine();
|
||||||
assertTrue(line.equals("Live Datanodes: 1"));
|
assertTrue(line.equals("Live Datanodes: 1"));
|
||||||
line = reader.readLine();
|
line = reader.readLine();
|
||||||
|
|
|
@ -603,6 +603,24 @@ public class TestSaveNamespace {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSaveNamespaceWithDanglingLease() throws Exception {
|
||||||
|
MiniDFSCluster cluster = new MiniDFSCluster.Builder(new Configuration())
|
||||||
|
.numDataNodes(1).build();
|
||||||
|
cluster.waitActive();
|
||||||
|
DistributedFileSystem fs = (DistributedFileSystem) cluster.getFileSystem();
|
||||||
|
try {
|
||||||
|
cluster.getNamesystem().leaseManager.addLease("me", "/non-existent");
|
||||||
|
fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
|
||||||
|
cluster.getNameNodeRpc().saveNamespace();
|
||||||
|
fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
|
||||||
|
} finally {
|
||||||
|
if (cluster != null) {
|
||||||
|
cluster.shutdown();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private void doAnEdit(FSNamesystem fsn, int id) throws IOException {
|
private void doAnEdit(FSNamesystem fsn, int id) throws IOException {
|
||||||
// Make an edit
|
// Make an edit
|
||||||
fsn.mkdirs(
|
fsn.mkdirs(
|
||||||
|
|
|
@ -430,10 +430,10 @@ public class TestSnapshotPathINodes {
|
||||||
// The number of inodes should be equal to components.length
|
// The number of inodes should be equal to components.length
|
||||||
assertEquals(newInodes.length, components.length);
|
assertEquals(newInodes.length, components.length);
|
||||||
// The last INode should be associated with file1
|
// The last INode should be associated with file1
|
||||||
assertEquals(newInodes[components.length - 1].getFullPathName(),
|
final int last = components.length - 1;
|
||||||
file1.toString());
|
assertEquals(newInodes[last].getFullPathName(), file1.toString());
|
||||||
// The modification time of the INode for file3 should have been changed
|
// The modification time of the INode for file3 should have been changed
|
||||||
Assert.assertFalse(inodes[components.length - 1].getModificationTime() ==
|
Assert.assertFalse(inodes[last].getModificationTime()
|
||||||
newInodes[components.length - 1].getModificationTime());
|
== newInodes[last].getModificationTime());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -221,7 +221,7 @@ public class TestINodeDirectoryWithSnapshot {
|
||||||
Assert.assertTrue(i >= 0);
|
Assert.assertTrue(i >= 0);
|
||||||
final INodeDirectory oldinode = (INodeDirectory)current.get(i);
|
final INodeDirectory oldinode = (INodeDirectory)current.get(i);
|
||||||
final INodeDirectory newinode = new INodeDirectory(oldinode);
|
final INodeDirectory newinode = new INodeDirectory(oldinode);
|
||||||
newinode.setModificationTime(oldinode.getModificationTime() + 1);
|
newinode.updateModificationTime(oldinode.getModificationTime() + 1);
|
||||||
|
|
||||||
current.set(i, newinode);
|
current.set(i, newinode);
|
||||||
if (diff != null) {
|
if (diff != null) {
|
||||||
|
|
|
@ -604,6 +604,9 @@ Release 0.23.6 - UNRELEASED
|
||||||
MAPREDUCE-4817. Hardcoded task ping timeout kills tasks localizing large
|
MAPREDUCE-4817. Hardcoded task ping timeout kills tasks localizing large
|
||||||
amounts of data (tgraves)
|
amounts of data (tgraves)
|
||||||
|
|
||||||
|
MAPREDUCE-4836. Elapsed time for running tasks on AM web UI tasks page is 0
|
||||||
|
(Ravi Prakash via jeagles)
|
||||||
|
|
||||||
Release 0.23.5 - UNRELEASED
|
Release 0.23.5 - UNRELEASED
|
||||||
|
|
||||||
INCOMPATIBLE CHANGES
|
INCOMPATIBLE CHANGES
|
||||||
|
|
|
@ -59,11 +59,12 @@ public class TaskInfo {
|
||||||
TaskReport report = task.getReport();
|
TaskReport report = task.getReport();
|
||||||
this.startTime = report.getStartTime();
|
this.startTime = report.getStartTime();
|
||||||
this.finishTime = report.getFinishTime();
|
this.finishTime = report.getFinishTime();
|
||||||
this.elapsedTime = Times.elapsed(this.startTime, this.finishTime, false);
|
this.state = report.getTaskState();
|
||||||
|
this.elapsedTime = Times.elapsed(this.startTime, this.finishTime,
|
||||||
|
this.state == TaskState.RUNNING);
|
||||||
if (this.elapsedTime == -1) {
|
if (this.elapsedTime == -1) {
|
||||||
this.elapsedTime = 0;
|
this.elapsedTime = 0;
|
||||||
}
|
}
|
||||||
this.state = report.getTaskState();
|
|
||||||
this.progress = report.getProgress() * 100;
|
this.progress = report.getProgress() * 100;
|
||||||
this.id = MRApps.toString(task.getID());
|
this.id = MRApps.toString(task.getID());
|
||||||
this.taskNum = task.getID().getId();
|
this.taskNum = task.getID().getId();
|
||||||
|
|
|
@ -117,7 +117,11 @@ Release 2.0.3-alpha - Unreleased
|
||||||
|
|
||||||
YARN-229. Remove old unused RM recovery code. (Bikas Saha via acmurthy)
|
YARN-229. Remove old unused RM recovery code. (Bikas Saha via acmurthy)
|
||||||
|
|
||||||
YARN-187. Add hierarchical queues to the fair scheduler. (Sandy Ryza via tomwhite)
|
YARN-187. Add hierarchical queues to the fair scheduler.
|
||||||
|
(Sandy Ryza via tomwhite)
|
||||||
|
|
||||||
|
YARN-72. NM should handle cleaning up containers when it shuts down.
|
||||||
|
(Sandy Ryza via tomwhite)
|
||||||
|
|
||||||
Release 2.0.2-alpha - 2012-09-07
|
Release 2.0.2-alpha - 2012-09-07
|
||||||
|
|
||||||
|
|
|
@ -25,13 +25,23 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
public class CMgrCompletedContainersEvent extends ContainerManagerEvent {
|
public class CMgrCompletedContainersEvent extends ContainerManagerEvent {
|
||||||
|
|
||||||
private List<ContainerId> containerToCleanup;
|
private List<ContainerId> containerToCleanup;
|
||||||
|
private Reason reason;
|
||||||
public CMgrCompletedContainersEvent(List<ContainerId> containersToCleanup) {
|
|
||||||
|
public CMgrCompletedContainersEvent(List<ContainerId> containersToCleanup, Reason reason) {
|
||||||
super(ContainerManagerEventType.FINISH_CONTAINERS);
|
super(ContainerManagerEventType.FINISH_CONTAINERS);
|
||||||
this.containerToCleanup = containersToCleanup;
|
this.containerToCleanup = containersToCleanup;
|
||||||
|
this.reason = reason;
|
||||||
}
|
}
|
||||||
|
|
||||||
public List<ContainerId> getContainersToCleanup() {
|
public List<ContainerId> getContainersToCleanup() {
|
||||||
return this.containerToCleanup;
|
return this.containerToCleanup;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Reason getReason() {
|
||||||
|
return reason;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static enum Reason {
|
||||||
|
ON_SHUTDOWN, BY_RESOURCEMANAGER
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,9 @@
|
||||||
package org.apache.hadoop.yarn.server.nodemanager;
|
package org.apache.hadoop.yarn.server.nodemanager;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
import java.util.concurrent.ConcurrentMap;
|
import java.util.concurrent.ConcurrentMap;
|
||||||
import java.util.concurrent.ConcurrentSkipListMap;
|
import java.util.concurrent.ConcurrentSkipListMap;
|
||||||
|
@ -61,14 +64,24 @@ public class NodeManager extends CompositeService implements
|
||||||
* Priority of the NodeManager shutdown hook.
|
* Priority of the NodeManager shutdown hook.
|
||||||
*/
|
*/
|
||||||
public static final int SHUTDOWN_HOOK_PRIORITY = 30;
|
public static final int SHUTDOWN_HOOK_PRIORITY = 30;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Extra duration to wait for containers to be killed on shutdown.
|
||||||
|
*/
|
||||||
|
private static final int SHUTDOWN_CLEANUP_SLOP_MS = 1000;
|
||||||
|
|
||||||
private static final Log LOG = LogFactory.getLog(NodeManager.class);
|
private static final Log LOG = LogFactory.getLog(NodeManager.class);
|
||||||
protected final NodeManagerMetrics metrics = NodeManagerMetrics.create();
|
protected final NodeManagerMetrics metrics = NodeManagerMetrics.create();
|
||||||
private ApplicationACLsManager aclsManager;
|
private ApplicationACLsManager aclsManager;
|
||||||
private NodeHealthCheckerService nodeHealthChecker;
|
private NodeHealthCheckerService nodeHealthChecker;
|
||||||
private LocalDirsHandlerService dirsHandler;
|
private LocalDirsHandlerService dirsHandler;
|
||||||
|
private Context context;
|
||||||
|
private AsyncDispatcher dispatcher;
|
||||||
|
private ContainerManagerImpl containerManager;
|
||||||
private static CompositeServiceShutdownHook nodeManagerShutdownHook;
|
private static CompositeServiceShutdownHook nodeManagerShutdownHook;
|
||||||
|
|
||||||
|
private long waitForContainersOnShutdownMillis;
|
||||||
|
|
||||||
public NodeManager() {
|
public NodeManager() {
|
||||||
super(NodeManager.class.getName());
|
super(NodeManager.class.getName());
|
||||||
}
|
}
|
||||||
|
@ -115,7 +128,7 @@ public class NodeManager extends CompositeService implements
|
||||||
containerTokenSecretManager = new NMContainerTokenSecretManager(conf);
|
containerTokenSecretManager = new NMContainerTokenSecretManager(conf);
|
||||||
}
|
}
|
||||||
|
|
||||||
Context context = new NMContext(containerTokenSecretManager);
|
this.context = new NMContext(containerTokenSecretManager);
|
||||||
|
|
||||||
this.aclsManager = new ApplicationACLsManager(conf);
|
this.aclsManager = new ApplicationACLsManager(conf);
|
||||||
|
|
||||||
|
@ -131,7 +144,7 @@ public class NodeManager extends CompositeService implements
|
||||||
addService(del);
|
addService(del);
|
||||||
|
|
||||||
// NodeManager level dispatcher
|
// NodeManager level dispatcher
|
||||||
AsyncDispatcher dispatcher = new AsyncDispatcher();
|
this.dispatcher = new AsyncDispatcher();
|
||||||
|
|
||||||
nodeHealthChecker = new NodeHealthCheckerService();
|
nodeHealthChecker = new NodeHealthCheckerService();
|
||||||
addService(nodeHealthChecker);
|
addService(nodeHealthChecker);
|
||||||
|
@ -144,7 +157,7 @@ public class NodeManager extends CompositeService implements
|
||||||
NodeResourceMonitor nodeResourceMonitor = createNodeResourceMonitor();
|
NodeResourceMonitor nodeResourceMonitor = createNodeResourceMonitor();
|
||||||
addService(nodeResourceMonitor);
|
addService(nodeResourceMonitor);
|
||||||
|
|
||||||
ContainerManagerImpl containerManager =
|
containerManager =
|
||||||
createContainerManager(context, exec, del, nodeStatusUpdater,
|
createContainerManager(context, exec, del, nodeStatusUpdater,
|
||||||
this.aclsManager, dirsHandler);
|
this.aclsManager, dirsHandler);
|
||||||
addService(containerManager);
|
addService(containerManager);
|
||||||
|
@ -155,13 +168,20 @@ public class NodeManager extends CompositeService implements
|
||||||
|
|
||||||
dispatcher.register(ContainerManagerEventType.class, containerManager);
|
dispatcher.register(ContainerManagerEventType.class, containerManager);
|
||||||
addService(dispatcher);
|
addService(dispatcher);
|
||||||
|
|
||||||
DefaultMetricsSystem.initialize("NodeManager");
|
DefaultMetricsSystem.initialize("NodeManager");
|
||||||
|
|
||||||
// StatusUpdater should be added last so that it get started last
|
// StatusUpdater should be added last so that it get started last
|
||||||
// so that we make sure everything is up before registering with RM.
|
// so that we make sure everything is up before registering with RM.
|
||||||
addService(nodeStatusUpdater);
|
addService(nodeStatusUpdater);
|
||||||
|
|
||||||
|
waitForContainersOnShutdownMillis =
|
||||||
|
conf.getLong(YarnConfiguration.NM_SLEEP_DELAY_BEFORE_SIGKILL_MS,
|
||||||
|
YarnConfiguration.DEFAULT_NM_SLEEP_DELAY_BEFORE_SIGKILL_MS) +
|
||||||
|
conf.getLong(YarnConfiguration.NM_PROCESS_KILL_WAIT_MS,
|
||||||
|
YarnConfiguration.DEFAULT_NM_PROCESS_KILL_WAIT_MS) +
|
||||||
|
SHUTDOWN_CLEANUP_SLOP_MS;
|
||||||
|
|
||||||
super.init(conf);
|
super.init(conf);
|
||||||
// TODO add local dirs to del
|
// TODO add local dirs to del
|
||||||
}
|
}
|
||||||
|
@ -178,9 +198,44 @@ public class NodeManager extends CompositeService implements
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void stop() {
|
public void stop() {
|
||||||
|
cleanupContainers();
|
||||||
super.stop();
|
super.stop();
|
||||||
DefaultMetricsSystem.shutdown();
|
DefaultMetricsSystem.shutdown();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
private void cleanupContainers() {
|
||||||
|
Map<ContainerId, Container> containers = context.getContainers();
|
||||||
|
if (containers.isEmpty()) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
LOG.info("Containers still running on shutdown: " + containers.keySet());
|
||||||
|
|
||||||
|
List<ContainerId> containerIds = new ArrayList<ContainerId>(containers.keySet());
|
||||||
|
dispatcher.getEventHandler().handle(
|
||||||
|
new CMgrCompletedContainersEvent(containerIds,
|
||||||
|
CMgrCompletedContainersEvent.Reason.ON_SHUTDOWN));
|
||||||
|
|
||||||
|
LOG.info("Waiting for containers to be killed");
|
||||||
|
|
||||||
|
long waitStartTime = System.currentTimeMillis();
|
||||||
|
while (!containers.isEmpty() &&
|
||||||
|
System.currentTimeMillis() - waitStartTime < waitForContainersOnShutdownMillis) {
|
||||||
|
try {
|
||||||
|
Thread.sleep(1000);
|
||||||
|
} catch (InterruptedException ex) {
|
||||||
|
LOG.warn("Interrupted while sleeping on container kill", ex);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// All containers killed
|
||||||
|
if (containers.isEmpty()) {
|
||||||
|
LOG.info("All containers in DONE state");
|
||||||
|
} else {
|
||||||
|
LOG.info("Done waiting for containers to be killed. Still alive: " +
|
||||||
|
containers.keySet());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public static class NMContext implements Context {
|
public static class NMContext implements Context {
|
||||||
|
|
||||||
|
@ -282,6 +337,11 @@ public class NodeManager extends CompositeService implements
|
||||||
NodeManager createNewNodeManager() {
|
NodeManager createNewNodeManager() {
|
||||||
return new NodeManager();
|
return new NodeManager();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// For testing
|
||||||
|
ContainerManagerImpl getContainerManager() {
|
||||||
|
return containerManager;
|
||||||
|
}
|
||||||
|
|
||||||
public static void main(String[] args) {
|
public static void main(String[] args) {
|
||||||
Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler());
|
Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler());
|
||||||
|
|
|
@ -363,7 +363,8 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
|
||||||
.getContainersToCleanupList();
|
.getContainersToCleanupList();
|
||||||
if (containersToCleanup.size() != 0) {
|
if (containersToCleanup.size() != 0) {
|
||||||
dispatcher.getEventHandler().handle(
|
dispatcher.getEventHandler().handle(
|
||||||
new CMgrCompletedContainersEvent(containersToCleanup));
|
new CMgrCompletedContainersEvent(containersToCleanup,
|
||||||
|
CMgrCompletedContainersEvent.Reason.BY_RESOURCEMANAGER));
|
||||||
}
|
}
|
||||||
List<ApplicationId> appsToCleanup =
|
List<ApplicationId> appsToCleanup =
|
||||||
response.getApplicationsToCleanupList();
|
response.getApplicationsToCleanupList();
|
||||||
|
|
|
@ -23,6 +23,8 @@ import static org.apache.hadoop.yarn.service.Service.STATE.STARTED;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.net.InetSocketAddress;
|
import java.net.InetSocketAddress;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
|
@ -593,9 +595,16 @@ public class ContainerManagerImpl extends CompositeService implements
|
||||||
(CMgrCompletedContainersEvent) event;
|
(CMgrCompletedContainersEvent) event;
|
||||||
for (ContainerId container : containersFinishedEvent
|
for (ContainerId container : containersFinishedEvent
|
||||||
.getContainersToCleanup()) {
|
.getContainersToCleanup()) {
|
||||||
|
String diagnostic = "";
|
||||||
|
if (containersFinishedEvent.getReason() ==
|
||||||
|
CMgrCompletedContainersEvent.Reason.ON_SHUTDOWN) {
|
||||||
|
diagnostic = "Container Killed on Shutdown";
|
||||||
|
} else if (containersFinishedEvent.getReason() ==
|
||||||
|
CMgrCompletedContainersEvent.Reason.BY_RESOURCEMANAGER) {
|
||||||
|
diagnostic = "Container Killed by ResourceManager";
|
||||||
|
}
|
||||||
this.dispatcher.getEventHandler().handle(
|
this.dispatcher.getEventHandler().handle(
|
||||||
new ContainerKillEvent(container,
|
new ContainerKillEvent(container, diagnostic));
|
||||||
"Container Killed by ResourceManager"));
|
|
||||||
}
|
}
|
||||||
break;
|
break;
|
||||||
default:
|
default:
|
||||||
|
|
|
@ -0,0 +1,92 @@
|
||||||
|
/**
|
||||||
|
* 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.yarn.server.nodemanager;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.yarn.event.Dispatcher;
|
||||||
|
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
|
||||||
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
||||||
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||||
|
import org.apache.hadoop.yarn.server.api.ResourceTracker;
|
||||||
|
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
|
||||||
|
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
|
||||||
|
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
|
||||||
|
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
|
||||||
|
import org.apache.hadoop.yarn.server.api.records.HeartbeatResponse;
|
||||||
|
import org.apache.hadoop.yarn.server.api.records.NodeStatus;
|
||||||
|
import org.apache.hadoop.yarn.server.api.records.RegistrationResponse;
|
||||||
|
import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This class allows a node manager to run without without communicating with a
|
||||||
|
* real RM.
|
||||||
|
*/
|
||||||
|
public class MockNodeStatusUpdater extends NodeStatusUpdaterImpl {
|
||||||
|
static final Log LOG = LogFactory.getLog(MockNodeStatusUpdater.class);
|
||||||
|
|
||||||
|
private static final RecordFactory recordFactory = RecordFactoryProvider
|
||||||
|
.getRecordFactory(null);
|
||||||
|
|
||||||
|
private ResourceTracker resourceTracker;
|
||||||
|
|
||||||
|
public MockNodeStatusUpdater(Context context, Dispatcher dispatcher,
|
||||||
|
NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) {
|
||||||
|
super(context, dispatcher, healthChecker, metrics);
|
||||||
|
resourceTracker = new MockResourceTracker();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected ResourceTracker getRMClient() {
|
||||||
|
return resourceTracker;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class MockResourceTracker implements ResourceTracker {
|
||||||
|
private int heartBeatID;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RegisterNodeManagerResponse registerNodeManager(
|
||||||
|
RegisterNodeManagerRequest request) throws YarnRemoteException {
|
||||||
|
RegistrationResponse regResponse = recordFactory
|
||||||
|
.newRecordInstance(RegistrationResponse.class);
|
||||||
|
|
||||||
|
RegisterNodeManagerResponse response = recordFactory
|
||||||
|
.newRecordInstance(RegisterNodeManagerResponse.class);
|
||||||
|
response.setRegistrationResponse(regResponse);
|
||||||
|
return response;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request)
|
||||||
|
throws YarnRemoteException {
|
||||||
|
NodeStatus nodeStatus = request.getNodeStatus();
|
||||||
|
LOG.info("Got heartbeat number " + heartBeatID);
|
||||||
|
nodeStatus.setResponseId(heartBeatID++);
|
||||||
|
|
||||||
|
HeartbeatResponse response = recordFactory
|
||||||
|
.newRecordInstance(HeartbeatResponse.class);
|
||||||
|
response.setResponseId(heartBeatID);
|
||||||
|
|
||||||
|
NodeHeartbeatResponse nhResponse = recordFactory
|
||||||
|
.newRecordInstance(NodeHeartbeatResponse.class);
|
||||||
|
nhResponse.setHeartbeatResponse(response);
|
||||||
|
return nhResponse;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,222 @@
|
||||||
|
/**
|
||||||
|
* 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.yarn.server.nodemanager;
|
||||||
|
|
||||||
|
import java.io.BufferedReader;
|
||||||
|
import java.io.BufferedWriter;
|
||||||
|
import java.io.File;
|
||||||
|
import java.io.FileReader;
|
||||||
|
import java.io.FileWriter;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import junit.framework.Assert;
|
||||||
|
|
||||||
|
import org.apache.hadoop.fs.FileContext;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.fs.UnsupportedFileSystemException;
|
||||||
|
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
|
||||||
|
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ContainerState;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
||||||
|
import org.apache.hadoop.yarn.api.records.LocalResource;
|
||||||
|
import org.apache.hadoop.yarn.api.records.LocalResourceType;
|
||||||
|
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
|
||||||
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
|
import org.apache.hadoop.yarn.api.records.URL;
|
||||||
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||||
|
import org.apache.hadoop.yarn.event.Dispatcher;
|
||||||
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
||||||
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||||
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
|
||||||
|
import org.apache.hadoop.yarn.util.ConverterUtils;
|
||||||
|
import org.junit.After;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
public class TestNodeManagerShutdown {
|
||||||
|
static final File basedir =
|
||||||
|
new File("target", TestNodeManagerShutdown.class.getName());
|
||||||
|
static final File tmpDir = new File(basedir, "tmpDir");
|
||||||
|
static final File logsDir = new File(basedir, "logs");
|
||||||
|
static final File remoteLogsDir = new File(basedir, "remotelogs");
|
||||||
|
static final File nmLocalDir = new File(basedir, "nm0");
|
||||||
|
static final File processStartFile = new File(tmpDir, "start_file.txt")
|
||||||
|
.getAbsoluteFile();
|
||||||
|
|
||||||
|
static final RecordFactory recordFactory = RecordFactoryProvider
|
||||||
|
.getRecordFactory(null);
|
||||||
|
static final String user = "nobody";
|
||||||
|
private FileContext localFS;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setup() throws UnsupportedFileSystemException {
|
||||||
|
localFS = FileContext.getLocalFSFileContext();
|
||||||
|
tmpDir.mkdirs();
|
||||||
|
logsDir.mkdirs();
|
||||||
|
remoteLogsDir.mkdirs();
|
||||||
|
nmLocalDir.mkdirs();
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void tearDown() throws IOException, InterruptedException {
|
||||||
|
localFS.delete(new Path(basedir.getPath()), true);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testKillContainersOnShutdown() throws IOException {
|
||||||
|
NodeManager nm = getNodeManager();
|
||||||
|
nm.init(createNMConfig());
|
||||||
|
nm.start();
|
||||||
|
|
||||||
|
ContainerManagerImpl containerManager = nm.getContainerManager();
|
||||||
|
File scriptFile = createUnhaltingScriptFile();
|
||||||
|
|
||||||
|
ContainerLaunchContext containerLaunchContext =
|
||||||
|
recordFactory.newRecordInstance(ContainerLaunchContext.class);
|
||||||
|
|
||||||
|
// Construct the Container-id
|
||||||
|
ContainerId cId = createContainerId();
|
||||||
|
containerLaunchContext.setContainerId(cId);
|
||||||
|
|
||||||
|
containerLaunchContext.setUser(user);
|
||||||
|
|
||||||
|
URL localResourceUri =
|
||||||
|
ConverterUtils.getYarnUrlFromPath(localFS
|
||||||
|
.makeQualified(new Path(scriptFile.getAbsolutePath())));
|
||||||
|
LocalResource localResource =
|
||||||
|
recordFactory.newRecordInstance(LocalResource.class);
|
||||||
|
localResource.setResource(localResourceUri);
|
||||||
|
localResource.setSize(-1);
|
||||||
|
localResource.setVisibility(LocalResourceVisibility.APPLICATION);
|
||||||
|
localResource.setType(LocalResourceType.FILE);
|
||||||
|
localResource.setTimestamp(scriptFile.lastModified());
|
||||||
|
String destinationFile = "dest_file";
|
||||||
|
Map<String, LocalResource> localResources =
|
||||||
|
new HashMap<String, LocalResource>();
|
||||||
|
localResources.put(destinationFile, localResource);
|
||||||
|
containerLaunchContext.setLocalResources(localResources);
|
||||||
|
containerLaunchContext.setUser(containerLaunchContext.getUser());
|
||||||
|
List<String> commands = new ArrayList<String>();
|
||||||
|
commands.add("/bin/bash");
|
||||||
|
commands.add(scriptFile.getAbsolutePath());
|
||||||
|
containerLaunchContext.setCommands(commands);
|
||||||
|
containerLaunchContext.setResource(recordFactory
|
||||||
|
.newRecordInstance(Resource.class));
|
||||||
|
containerLaunchContext.getResource().setMemory(1024);
|
||||||
|
StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
|
||||||
|
startRequest.setContainerLaunchContext(containerLaunchContext);
|
||||||
|
containerManager.startContainer(startRequest);
|
||||||
|
|
||||||
|
GetContainerStatusRequest request =
|
||||||
|
recordFactory.newRecordInstance(GetContainerStatusRequest.class);
|
||||||
|
request.setContainerId(cId);
|
||||||
|
ContainerStatus containerStatus =
|
||||||
|
containerManager.getContainerStatus(request).getStatus();
|
||||||
|
Assert.assertEquals(ContainerState.RUNNING, containerStatus.getState());
|
||||||
|
|
||||||
|
try {Thread.sleep(5000);} catch (InterruptedException ex) {ex.printStackTrace();}
|
||||||
|
|
||||||
|
nm.stop();
|
||||||
|
|
||||||
|
// Now verify the contents of the file
|
||||||
|
// Script generates a message when it receives a sigterm
|
||||||
|
// so we look for that
|
||||||
|
BufferedReader reader =
|
||||||
|
new BufferedReader(new FileReader(processStartFile));
|
||||||
|
|
||||||
|
boolean foundSigTermMessage = false;
|
||||||
|
while (true) {
|
||||||
|
String line = reader.readLine();
|
||||||
|
if (line == null) {
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
if (line.contains("SIGTERM")) {
|
||||||
|
foundSigTermMessage = true;
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
Assert.assertTrue("Did not find sigterm message", foundSigTermMessage);
|
||||||
|
reader.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
private ContainerId createContainerId() {
|
||||||
|
ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class);
|
||||||
|
appId.setClusterTimestamp(0);
|
||||||
|
appId.setId(0);
|
||||||
|
ApplicationAttemptId appAttemptId =
|
||||||
|
recordFactory.newRecordInstance(ApplicationAttemptId.class);
|
||||||
|
appAttemptId.setApplicationId(appId);
|
||||||
|
appAttemptId.setAttemptId(1);
|
||||||
|
ContainerId containerId =
|
||||||
|
recordFactory.newRecordInstance(ContainerId.class);
|
||||||
|
containerId.setApplicationAttemptId(appAttemptId);
|
||||||
|
return containerId;
|
||||||
|
}
|
||||||
|
|
||||||
|
private YarnConfiguration createNMConfig() {
|
||||||
|
YarnConfiguration conf = new YarnConfiguration();
|
||||||
|
conf.setInt(YarnConfiguration.NM_PMEM_MB, 5*1024); // 5GB
|
||||||
|
conf.set(YarnConfiguration.NM_ADDRESS, "127.0.0.1:12345");
|
||||||
|
conf.set(YarnConfiguration.NM_LOCALIZER_ADDRESS, "127.0.0.1:12346");
|
||||||
|
conf.set(YarnConfiguration.NM_LOG_DIRS, logsDir.getAbsolutePath());
|
||||||
|
conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogsDir.getAbsolutePath());
|
||||||
|
conf.set(YarnConfiguration.NM_LOCAL_DIRS, nmLocalDir.getAbsolutePath());
|
||||||
|
return conf;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates a script to run a container that will run forever unless
|
||||||
|
* stopped by external means.
|
||||||
|
*/
|
||||||
|
private File createUnhaltingScriptFile() throws IOException {
|
||||||
|
File scriptFile = new File(tmpDir, "scriptFile.sh");
|
||||||
|
BufferedWriter fileWriter = new BufferedWriter(new FileWriter(scriptFile));
|
||||||
|
fileWriter.write("#!/bin/bash\n\n");
|
||||||
|
fileWriter.write("echo \"Running testscript for delayed kill\"\n");
|
||||||
|
fileWriter.write("hello=\"Got SIGTERM\"\n");
|
||||||
|
fileWriter.write("umask 0\n");
|
||||||
|
fileWriter.write("trap \"echo $hello >> " + processStartFile + "\" SIGTERM\n");
|
||||||
|
fileWriter.write("echo \"Writing pid to start file\"\n");
|
||||||
|
fileWriter.write("echo $$ >> " + processStartFile + "\n");
|
||||||
|
fileWriter.write("while true; do\nsleep 1s;\ndone\n");
|
||||||
|
|
||||||
|
fileWriter.close();
|
||||||
|
return scriptFile;
|
||||||
|
}
|
||||||
|
|
||||||
|
private NodeManager getNodeManager() {
|
||||||
|
return new NodeManager() {
|
||||||
|
@Override
|
||||||
|
protected NodeStatusUpdater createNodeStatusUpdater(Context context,
|
||||||
|
Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
|
||||||
|
MockNodeStatusUpdater myNodeStatusUpdater = new MockNodeStatusUpdater(
|
||||||
|
context, dispatcher, healthChecker, metrics);
|
||||||
|
return myNodeStatusUpdater;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue