HADOOP-10715. Remove public GraphiteSink#setWriter()
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1605646 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
9571db19eb
commit
7fcaab2350
@ -646,6 +646,8 @@ Release 2.5.0 - UNRELEASED
|
|||||||
HADOOP-10479. Fix new findbugs warnings in hadoop-minikdc.
|
HADOOP-10479. Fix new findbugs warnings in hadoop-minikdc.
|
||||||
(Swarnim Kulkarni via wheat9)
|
(Swarnim Kulkarni via wheat9)
|
||||||
|
|
||||||
|
HADOOP-10715. Remove public GraphiteSink#setWriter (Babak Behzad via raviprak)
|
||||||
|
|
||||||
Release 2.4.1 - 2014-06-23
|
Release 2.4.1 - 2014-06-23
|
||||||
|
|
||||||
INCOMPATIBLE CHANGES
|
INCOMPATIBLE CHANGES
|
||||||
|
@ -50,10 +50,6 @@ public class GraphiteSink implements MetricsSink, Closeable {
|
|||||||
private String metricsPrefix = null;
|
private String metricsPrefix = null;
|
||||||
private Socket socket = null;
|
private Socket socket = null;
|
||||||
|
|
||||||
public void setWriter(Writer writer) {
|
|
||||||
this.writer = writer;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void init(SubsetConfiguration conf) {
|
public void init(SubsetConfiguration conf) {
|
||||||
// Get Graphite host configurations.
|
// Get Graphite host configurations.
|
||||||
@ -68,7 +64,7 @@ public void init(SubsetConfiguration conf) {
|
|||||||
try {
|
try {
|
||||||
// Open an connection to Graphite server.
|
// Open an connection to Graphite server.
|
||||||
socket = new Socket(serverHost, serverPort);
|
socket = new Socket(serverHost, serverPort);
|
||||||
setWriter(new OutputStreamWriter(socket.getOutputStream()));
|
writer = new OutputStreamWriter(socket.getOutputStream());
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
throw new MetricsException("Error creating connection, "
|
throw new MetricsException("Error creating connection, "
|
||||||
+ serverHost + ":" + serverPort, e);
|
+ serverHost + ":" + serverPort, e);
|
||||||
|
@ -28,15 +28,16 @@
|
|||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
import static org.mockito.Mockito.*;
|
|
||||||
|
|
||||||
import org.apache.hadoop.metrics2.AbstractMetric;
|
import org.apache.hadoop.metrics2.AbstractMetric;
|
||||||
import org.apache.hadoop.metrics2.MetricsException;
|
import org.apache.hadoop.metrics2.MetricsException;
|
||||||
import org.apache.hadoop.metrics2.MetricsRecord;
|
import org.apache.hadoop.metrics2.MetricsRecord;
|
||||||
import org.apache.hadoop.metrics2.MetricsTag;
|
import org.apache.hadoop.metrics2.MetricsTag;
|
||||||
import org.apache.hadoop.metrics2.sink.GraphiteSink;
|
import org.apache.hadoop.metrics2.sink.GraphiteSink;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import static org.mockito.Mockito.*;
|
||||||
import org.mockito.ArgumentCaptor;
|
import org.mockito.ArgumentCaptor;
|
||||||
|
import org.mockito.internal.util.reflection.Whitebox;
|
||||||
|
|
||||||
public class TestGraphiteMetrics {
|
public class TestGraphiteMetrics {
|
||||||
private AbstractMetric makeMetric(String name, Number value) {
|
private AbstractMetric makeMetric(String name, Number value) {
|
||||||
@ -57,14 +58,13 @@ public void testPutMetrics() {
|
|||||||
metrics.add(makeMetric("foo2", 2.25));
|
metrics.add(makeMetric("foo2", 2.25));
|
||||||
MetricsRecord record = new MetricsRecordImpl(MsInfo.Context, (long) 10000, tags, metrics);
|
MetricsRecord record = new MetricsRecordImpl(MsInfo.Context, (long) 10000, tags, metrics);
|
||||||
|
|
||||||
OutputStreamWriter writer = mock(OutputStreamWriter.class);
|
OutputStreamWriter mockWriter = mock(OutputStreamWriter.class);
|
||||||
ArgumentCaptor<String> argument = ArgumentCaptor.forClass(String.class);
|
ArgumentCaptor<String> argument = ArgumentCaptor.forClass(String.class);
|
||||||
|
Whitebox.setInternalState(sink, "writer", mockWriter);
|
||||||
sink.setWriter(writer);
|
|
||||||
sink.putMetrics(record);
|
sink.putMetrics(record);
|
||||||
|
|
||||||
try {
|
try {
|
||||||
verify(writer).write(argument.capture());
|
verify(mockWriter).write(argument.capture());
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
e.printStackTrace();
|
e.printStackTrace();
|
||||||
}
|
}
|
||||||
@ -89,14 +89,13 @@ public void testPutMetrics2() {
|
|||||||
metrics.add(makeMetric("foo2", 2));
|
metrics.add(makeMetric("foo2", 2));
|
||||||
MetricsRecord record = new MetricsRecordImpl(MsInfo.Context, (long) 10000, tags, metrics);
|
MetricsRecord record = new MetricsRecordImpl(MsInfo.Context, (long) 10000, tags, metrics);
|
||||||
|
|
||||||
OutputStreamWriter writer = mock(OutputStreamWriter.class);
|
OutputStreamWriter mockWriter = mock(OutputStreamWriter.class);
|
||||||
ArgumentCaptor<String> argument = ArgumentCaptor.forClass(String.class);
|
ArgumentCaptor<String> argument = ArgumentCaptor.forClass(String.class);
|
||||||
|
Whitebox.setInternalState(sink, "writer", mockWriter);
|
||||||
sink.setWriter(writer);
|
|
||||||
sink.putMetrics(record);
|
sink.putMetrics(record);
|
||||||
|
|
||||||
try {
|
try {
|
||||||
verify(writer).write(argument.capture());
|
verify(mockWriter).write(argument.capture());
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
e.printStackTrace();
|
e.printStackTrace();
|
||||||
}
|
}
|
||||||
@ -122,26 +121,26 @@ public void testCloseAndWrite() throws IOException {
|
|||||||
|
|
||||||
OutputStreamWriter writer = mock(OutputStreamWriter.class);
|
OutputStreamWriter writer = mock(OutputStreamWriter.class);
|
||||||
|
|
||||||
sink.setWriter(writer);
|
Whitebox.setInternalState(sink, "writer", writer);
|
||||||
sink.close();
|
sink.close();
|
||||||
sink.putMetrics(record);
|
sink.putMetrics(record);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testClose(){
|
public void testClose(){
|
||||||
GraphiteSink sink = new GraphiteSink();
|
GraphiteSink sink = new GraphiteSink();
|
||||||
Writer mockWriter = mock(Writer.class);
|
Writer mockWriter = mock(Writer.class);
|
||||||
sink.setWriter(mockWriter);
|
Whitebox.setInternalState(sink, "writer", mockWriter);
|
||||||
try {
|
try {
|
||||||
sink.close();
|
sink.close();
|
||||||
} catch (IOException ioe) {
|
} catch (IOException ioe) {
|
||||||
ioe.printStackTrace();
|
ioe.printStackTrace();
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
verify(mockWriter).close();
|
verify(mockWriter).close();
|
||||||
} catch (IOException ioe) {
|
} catch (IOException ioe) {
|
||||||
ioe.printStackTrace();
|
ioe.printStackTrace();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user