HADOOP-11971. Move test utilities for tracing from hadoop-hdfs to hadoop-common. Contributed by Masatake Iwasaki.
(cherry picked from commit e4489d97e5
)
This commit is contained in:
parent
3863342ca9
commit
e397cca456
|
@ -152,6 +152,9 @@ Release 2.8.0 - UNRELEASED
|
||||||
|
|
||||||
HADOOP-12055. Deprecate usage of NativeIO#link. (Andrew Wang via cnauroth)
|
HADOOP-12055. Deprecate usage of NativeIO#link. (Andrew Wang via cnauroth)
|
||||||
|
|
||||||
|
HADOOP-11971. Move test utilities for tracing from hadoop-hdfs to
|
||||||
|
hadoop-common. (Masatake Iwasaki via aajisaka)
|
||||||
|
|
||||||
OPTIMIZATIONS
|
OPTIMIZATIONS
|
||||||
|
|
||||||
HADOOP-11785. Reduce the number of listStatus operation in distcp
|
HADOOP-11785. Reduce the number of listStatus operation in distcp
|
||||||
|
|
|
@ -0,0 +1,109 @@
|
||||||
|
/**
|
||||||
|
* 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.tracing;
|
||||||
|
|
||||||
|
import com.google.common.base.Supplier;
|
||||||
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
|
import org.apache.htrace.Span;
|
||||||
|
import org.apache.htrace.SpanReceiver;
|
||||||
|
import org.apache.htrace.HTraceConfiguration;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.LinkedList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
|
import java.util.concurrent.TimeoutException;
|
||||||
|
import org.junit.Assert;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Span receiver that puts all spans into a single set.
|
||||||
|
* This is useful for testing.
|
||||||
|
* <p/>
|
||||||
|
* We're not using HTrace's POJOReceiver here so as that doesn't
|
||||||
|
* push all the metrics to a static place, and would make testing
|
||||||
|
* SpanReceiverHost harder.
|
||||||
|
*/
|
||||||
|
public class SetSpanReceiver implements SpanReceiver {
|
||||||
|
|
||||||
|
public SetSpanReceiver(HTraceConfiguration conf) {
|
||||||
|
}
|
||||||
|
|
||||||
|
public void receiveSpan(Span span) {
|
||||||
|
SetHolder.spans.put(span.getSpanId(), span);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void close() {
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void clear() {
|
||||||
|
SetHolder.spans.clear();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static int size() {
|
||||||
|
return SetHolder.spans.size();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Collection<Span> getSpans() {
|
||||||
|
return SetHolder.spans.values();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Map<String, List<Span>> getMap() {
|
||||||
|
return SetHolder.getMap();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class SetHolder {
|
||||||
|
public static ConcurrentHashMap<Long, Span> spans =
|
||||||
|
new ConcurrentHashMap<Long, Span>();
|
||||||
|
|
||||||
|
public static Map<String, List<Span>> getMap() {
|
||||||
|
Map<String, List<Span>> map = new HashMap<String, List<Span>>();
|
||||||
|
|
||||||
|
for (Span s : spans.values()) {
|
||||||
|
List<Span> l = map.get(s.getDescription());
|
||||||
|
if (l == null) {
|
||||||
|
l = new LinkedList<Span>();
|
||||||
|
map.put(s.getDescription(), l);
|
||||||
|
}
|
||||||
|
l.add(s);
|
||||||
|
}
|
||||||
|
return map;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static void assertSpanNamesFound(final String[] expectedSpanNames) {
|
||||||
|
try {
|
||||||
|
GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
||||||
|
@Override
|
||||||
|
public Boolean get() {
|
||||||
|
Map<String, List<Span>> map = SetSpanReceiver.SetHolder.getMap();
|
||||||
|
for (String spanName : expectedSpanNames) {
|
||||||
|
if (!map.containsKey(spanName)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}, 100, 1000);
|
||||||
|
} catch (TimeoutException e) {
|
||||||
|
Assert.fail("timed out to get expected spans: " + e.getMessage());
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
Assert.fail("interrupted while waiting spans: " + e.getMessage());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -25,11 +25,8 @@ import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||||
import org.apache.hadoop.test.GenericTestUtils;
|
|
||||||
import org.apache.htrace.HTraceConfiguration;
|
|
||||||
import org.apache.htrace.Sampler;
|
import org.apache.htrace.Sampler;
|
||||||
import org.apache.htrace.Span;
|
import org.apache.htrace.Span;
|
||||||
import org.apache.htrace.SpanReceiver;
|
|
||||||
import org.apache.htrace.Trace;
|
import org.apache.htrace.Trace;
|
||||||
import org.apache.htrace.TraceScope;
|
import org.apache.htrace.TraceScope;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
|
@ -37,17 +34,10 @@ import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.BeforeClass;
|
import org.junit.BeforeClass;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.LinkedList;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
|
||||||
import java.util.concurrent.TimeoutException;
|
|
||||||
|
|
||||||
import com.google.common.base.Supplier;
|
|
||||||
|
|
||||||
public class TestTracing {
|
public class TestTracing {
|
||||||
|
|
||||||
|
@ -60,9 +50,9 @@ public class TestTracing {
|
||||||
// write and read without tracing started
|
// write and read without tracing started
|
||||||
String fileName = "testTracingDisabled.dat";
|
String fileName = "testTracingDisabled.dat";
|
||||||
writeTestFile(fileName);
|
writeTestFile(fileName);
|
||||||
Assert.assertTrue(SetSpanReceiver.SetHolder.size() == 0);
|
Assert.assertTrue(SetSpanReceiver.size() == 0);
|
||||||
readTestFile(fileName);
|
readTestFile(fileName);
|
||||||
Assert.assertTrue(SetSpanReceiver.SetHolder.size() == 0);
|
Assert.assertTrue(SetSpanReceiver.size() == 0);
|
||||||
|
|
||||||
writeWithTracing();
|
writeWithTracing();
|
||||||
readWithTracing();
|
readWithTracing();
|
||||||
|
@ -91,10 +81,10 @@ public class TestTracing {
|
||||||
"org.apache.hadoop.hdfs.protocol.ClientProtocol.addBlock",
|
"org.apache.hadoop.hdfs.protocol.ClientProtocol.addBlock",
|
||||||
"ClientNamenodeProtocol#addBlock"
|
"ClientNamenodeProtocol#addBlock"
|
||||||
};
|
};
|
||||||
assertSpanNamesFound(expectedSpanNames);
|
SetSpanReceiver.assertSpanNamesFound(expectedSpanNames);
|
||||||
|
|
||||||
// The trace should last about the same amount of time as the test
|
// The trace should last about the same amount of time as the test
|
||||||
Map<String, List<Span>> map = SetSpanReceiver.SetHolder.getMap();
|
Map<String, List<Span>> map = SetSpanReceiver.getMap();
|
||||||
Span s = map.get("testWriteTraceHooks").get(0);
|
Span s = map.get("testWriteTraceHooks").get(0);
|
||||||
Assert.assertNotNull(s);
|
Assert.assertNotNull(s);
|
||||||
long spanStart = s.getStartTimeMillis();
|
long spanStart = s.getStartTimeMillis();
|
||||||
|
@ -127,7 +117,7 @@ public class TestTracing {
|
||||||
.get(0).getTimelineAnnotations()
|
.get(0).getTimelineAnnotations()
|
||||||
.get(0).getMessage());
|
.get(0).getMessage());
|
||||||
|
|
||||||
SetSpanReceiver.SetHolder.spans.clear();
|
SetSpanReceiver.clear();
|
||||||
}
|
}
|
||||||
|
|
||||||
public void readWithTracing() throws Exception {
|
public void readWithTracing() throws Exception {
|
||||||
|
@ -145,10 +135,10 @@ public class TestTracing {
|
||||||
"ClientNamenodeProtocol#getBlockLocations",
|
"ClientNamenodeProtocol#getBlockLocations",
|
||||||
"OpReadBlockProto"
|
"OpReadBlockProto"
|
||||||
};
|
};
|
||||||
assertSpanNamesFound(expectedSpanNames);
|
SetSpanReceiver.assertSpanNamesFound(expectedSpanNames);
|
||||||
|
|
||||||
// The trace should last about the same amount of time as the test
|
// The trace should last about the same amount of time as the test
|
||||||
Map<String, List<Span>> map = SetSpanReceiver.SetHolder.getMap();
|
Map<String, List<Span>> map = SetSpanReceiver.getMap();
|
||||||
Span s = map.get("testReadTraceHooks").get(0);
|
Span s = map.get("testReadTraceHooks").get(0);
|
||||||
Assert.assertNotNull(s);
|
Assert.assertNotNull(s);
|
||||||
|
|
||||||
|
@ -159,10 +149,10 @@ public class TestTracing {
|
||||||
|
|
||||||
// There should only be one trace id as it should all be homed in the
|
// There should only be one trace id as it should all be homed in the
|
||||||
// top trace.
|
// top trace.
|
||||||
for (Span span : SetSpanReceiver.SetHolder.spans.values()) {
|
for (Span span : SetSpanReceiver.getSpans()) {
|
||||||
Assert.assertEquals(ts.getSpan().getTraceId(), span.getTraceId());
|
Assert.assertEquals(ts.getSpan().getTraceId(), span.getTraceId());
|
||||||
}
|
}
|
||||||
SetSpanReceiver.SetHolder.spans.clear();
|
SetSpanReceiver.clear();
|
||||||
}
|
}
|
||||||
|
|
||||||
private void writeTestFile(String testFileName) throws Exception {
|
private void writeTestFile(String testFileName) throws Exception {
|
||||||
|
@ -211,7 +201,7 @@ public class TestTracing {
|
||||||
.build();
|
.build();
|
||||||
cluster.waitActive();
|
cluster.waitActive();
|
||||||
dfs = cluster.getFileSystem();
|
dfs = cluster.getFileSystem();
|
||||||
SetSpanReceiver.SetHolder.spans.clear();
|
SetSpanReceiver.clear();
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
|
@ -219,68 +209,4 @@ public class TestTracing {
|
||||||
cluster.shutdown();
|
cluster.shutdown();
|
||||||
}
|
}
|
||||||
|
|
||||||
static void assertSpanNamesFound(final String[] expectedSpanNames) {
|
|
||||||
try {
|
|
||||||
GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
||||||
@Override
|
|
||||||
public Boolean get() {
|
|
||||||
Map<String, List<Span>> map = SetSpanReceiver.SetHolder.getMap();
|
|
||||||
for (String spanName : expectedSpanNames) {
|
|
||||||
if (!map.containsKey(spanName)) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
}, 100, 1000);
|
|
||||||
} catch (TimeoutException e) {
|
|
||||||
Assert.fail("timed out to get expected spans: " + e.getMessage());
|
|
||||||
} catch (InterruptedException e) {
|
|
||||||
Assert.fail("interrupted while waiting spans: " + e.getMessage());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Span receiver that puts all spans into a single set.
|
|
||||||
* This is useful for testing.
|
|
||||||
* <p/>
|
|
||||||
* We're not using HTrace's POJOReceiver here so as that doesn't
|
|
||||||
* push all the metrics to a static place, and would make testing
|
|
||||||
* SpanReceiverHost harder.
|
|
||||||
*/
|
|
||||||
public static class SetSpanReceiver implements SpanReceiver {
|
|
||||||
|
|
||||||
public SetSpanReceiver(HTraceConfiguration conf) {
|
|
||||||
}
|
|
||||||
|
|
||||||
public void receiveSpan(Span span) {
|
|
||||||
SetHolder.spans.put(span.getSpanId(), span);
|
|
||||||
}
|
|
||||||
|
|
||||||
public void close() {
|
|
||||||
}
|
|
||||||
|
|
||||||
public static class SetHolder {
|
|
||||||
public static ConcurrentHashMap<Long, Span> spans =
|
|
||||||
new ConcurrentHashMap<Long, Span>();
|
|
||||||
|
|
||||||
public static int size() {
|
|
||||||
return spans.size();
|
|
||||||
}
|
|
||||||
|
|
||||||
public static Map<String, List<Span>> getMap() {
|
|
||||||
Map<String, List<Span>> map = new HashMap<String, List<Span>>();
|
|
||||||
|
|
||||||
for (Span s : spans.values()) {
|
|
||||||
List<Span> l = map.get(s.getDescription());
|
|
||||||
if (l == null) {
|
|
||||||
l = new LinkedList<Span>();
|
|
||||||
map.put(s.getDescription(), l);
|
|
||||||
}
|
|
||||||
l.add(s);
|
|
||||||
}
|
|
||||||
return map;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -66,7 +66,7 @@ public class TestTracingShortCircuitLocalRead {
|
||||||
conf = new Configuration();
|
conf = new Configuration();
|
||||||
conf.set(DFSConfigKeys.DFS_CLIENT_HTRACE_PREFIX +
|
conf.set(DFSConfigKeys.DFS_CLIENT_HTRACE_PREFIX +
|
||||||
SpanReceiverHost.SPAN_RECEIVERS_CONF_SUFFIX,
|
SpanReceiverHost.SPAN_RECEIVERS_CONF_SUFFIX,
|
||||||
TestTracing.SetSpanReceiver.class.getName());
|
SetSpanReceiver.class.getName());
|
||||||
conf.setLong("dfs.blocksize", 100 * 1024);
|
conf.setLong("dfs.blocksize", 100 * 1024);
|
||||||
conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
|
conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
|
||||||
conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.SKIP_CHECKSUM_KEY, false);
|
conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.SKIP_CHECKSUM_KEY, false);
|
||||||
|
@ -92,7 +92,7 @@ public class TestTracingShortCircuitLocalRead {
|
||||||
"OpRequestShortCircuitAccessProto",
|
"OpRequestShortCircuitAccessProto",
|
||||||
"ShortCircuitShmRequestProto"
|
"ShortCircuitShmRequestProto"
|
||||||
};
|
};
|
||||||
TestTracing.assertSpanNamesFound(expectedSpanNames);
|
SetSpanReceiver.assertSpanNamesFound(expectedSpanNames);
|
||||||
} finally {
|
} finally {
|
||||||
dfs.close();
|
dfs.close();
|
||||||
cluster.shutdown();
|
cluster.shutdown();
|
||||||
|
|
Loading…
Reference in New Issue