HADOOP-18446. [SBN read] Add a re-queue metric to RpcMetrics to quantify the number of re-queued RPCs (#4871)
Signed-off-by: Erik Krogen <xkrogen@apache.org> Co-authored-by: zengqiang.xu <zengqiang.xu@shopee.com>
This commit is contained in:
parent
0f03299eba
commit
a73c4804d8
@ -3156,6 +3156,7 @@ private void requeueCall(Call call)
|
|||||||
throws IOException, InterruptedException {
|
throws IOException, InterruptedException {
|
||||||
try {
|
try {
|
||||||
internalQueueCall(call, false);
|
internalQueueCall(call, false);
|
||||||
|
rpcMetrics.incrRequeueCalls();
|
||||||
} catch (RpcServerException rse) {
|
} catch (RpcServerException rse) {
|
||||||
call.doResponse(rse.getCause(), rse.getRpcStatusProto());
|
call.doResponse(rse.getCause(), rse.getRpcStatusProto());
|
||||||
}
|
}
|
||||||
|
@ -128,6 +128,8 @@ public static RpcMetrics create(Server server, Configuration conf) {
|
|||||||
MutableCounterLong rpcClientBackoff;
|
MutableCounterLong rpcClientBackoff;
|
||||||
@Metric("Number of Slow RPC calls")
|
@Metric("Number of Slow RPC calls")
|
||||||
MutableCounterLong rpcSlowCalls;
|
MutableCounterLong rpcSlowCalls;
|
||||||
|
@Metric("Number of requeue calls")
|
||||||
|
MutableCounterLong rpcRequeueCalls;
|
||||||
|
|
||||||
@Metric("Number of open connections") public int numOpenConnections() {
|
@Metric("Number of open connections") public int numOpenConnections() {
|
||||||
return server.getNumOpenConnections();
|
return server.getNumOpenConnections();
|
||||||
@ -304,6 +306,13 @@ public void incrSlowRpc() {
|
|||||||
rpcSlowCalls.incr();
|
rpcSlowCalls.incr();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Increments the Requeue Calls counter.
|
||||||
|
*/
|
||||||
|
public void incrRequeueCalls() {
|
||||||
|
rpcRequeueCalls.incr();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns a MutableRate Counter.
|
* Returns a MutableRate Counter.
|
||||||
* @return Mutable Rate
|
* @return Mutable Rate
|
||||||
@ -344,6 +353,15 @@ public long getRpcSlowCalls() {
|
|||||||
return rpcSlowCalls.value();
|
return rpcSlowCalls.value();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the number of requeue calls.
|
||||||
|
* @return long
|
||||||
|
*/
|
||||||
|
@VisibleForTesting
|
||||||
|
public long getRpcRequeueCalls() {
|
||||||
|
return rpcRequeueCalls.value();
|
||||||
|
}
|
||||||
|
|
||||||
public MutableRate getDeferredRpcProcessingTime() {
|
public MutableRate getDeferredRpcProcessingTime() {
|
||||||
return deferredRpcProcessingTime;
|
return deferredRpcProcessingTime;
|
||||||
}
|
}
|
||||||
|
@ -21,6 +21,7 @@
|
|||||||
import static org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter.getServiceState;
|
import static org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter.getServiceState;
|
||||||
import static org.apache.hadoop.hdfs.server.namenode.ha.ObserverReadProxyProvider.*;
|
import static org.apache.hadoop.hdfs.server.namenode.ha.ObserverReadProxyProvider.*;
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertNotNull;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
import static org.junit.Assert.fail;
|
import static org.junit.Assert.fail;
|
||||||
import static org.mockito.ArgumentMatchers.any;
|
import static org.mockito.ArgumentMatchers.any;
|
||||||
@ -36,7 +37,10 @@
|
|||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
|
import java.util.concurrent.Executors;
|
||||||
import java.util.concurrent.Future;
|
import java.util.concurrent.Future;
|
||||||
|
import java.util.concurrent.ScheduledExecutorService;
|
||||||
|
import java.util.concurrent.ScheduledFuture;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
@ -61,9 +65,12 @@
|
|||||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
||||||
|
import org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.TestFsck;
|
import org.apache.hadoop.hdfs.server.namenode.TestFsck;
|
||||||
import org.apache.hadoop.hdfs.tools.GetGroups;
|
import org.apache.hadoop.hdfs.tools.GetGroups;
|
||||||
import org.apache.hadoop.ipc.ObserverRetryOnActiveException;
|
import org.apache.hadoop.ipc.ObserverRetryOnActiveException;
|
||||||
|
import org.apache.hadoop.ipc.metrics.RpcMetrics;
|
||||||
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
import org.apache.hadoop.util.Time;
|
import org.apache.hadoop.util.Time;
|
||||||
import org.apache.hadoop.util.concurrent.HadoopExecutors;
|
import org.apache.hadoop.util.concurrent.HadoopExecutors;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
@ -124,6 +131,43 @@ public static void shutDownCluster() throws IOException {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testObserverRequeue() throws Exception {
|
||||||
|
ScheduledExecutorService interruptor =
|
||||||
|
Executors.newScheduledThreadPool(1);
|
||||||
|
|
||||||
|
FSNamesystem observerFsNS = dfsCluster.getNamesystem(2);
|
||||||
|
RpcMetrics obRpcMetrics = ((NameNodeRpcServer)dfsCluster
|
||||||
|
.getNameNodeRpc(2)).getClientRpcServer().getRpcMetrics();
|
||||||
|
try {
|
||||||
|
// Stop EditlogTailer of Observer NameNode.
|
||||||
|
observerFsNS.getEditLogTailer().stop();
|
||||||
|
long oldRequeueNum = obRpcMetrics.getRpcRequeueCalls();
|
||||||
|
ScheduledFuture<FileStatus> scheduledFuture = interruptor.schedule(
|
||||||
|
() -> {
|
||||||
|
Path tmpTestPath = new Path("/TestObserverRequeue");
|
||||||
|
dfs.create(tmpTestPath, (short)1).close();
|
||||||
|
assertSentTo(0);
|
||||||
|
// This operation will be blocked in ObserverNameNode
|
||||||
|
// until EditlogTailer tailed edits from journalNode.
|
||||||
|
FileStatus fileStatus = dfs.getFileStatus(tmpTestPath);
|
||||||
|
assertSentTo(2);
|
||||||
|
return fileStatus;
|
||||||
|
}, 0, TimeUnit.SECONDS);
|
||||||
|
|
||||||
|
GenericTestUtils.waitFor(() -> obRpcMetrics.getRpcRequeueCalls() > oldRequeueNum,
|
||||||
|
50, 10000);
|
||||||
|
|
||||||
|
observerFsNS.getEditLogTailer().doTailEdits();
|
||||||
|
FileStatus fileStatus = scheduledFuture.get(10000, TimeUnit.MILLISECONDS);
|
||||||
|
assertNotNull(fileStatus);
|
||||||
|
} finally {
|
||||||
|
EditLogTailer editLogTailer = new EditLogTailer(observerFsNS, conf);
|
||||||
|
observerFsNS.setEditLogTailerForTests(editLogTailer);
|
||||||
|
editLogTailer.start();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testNoActiveToObserver() throws Exception {
|
public void testNoActiveToObserver() throws Exception {
|
||||||
try {
|
try {
|
||||||
|
Loading…
Reference in New Issue
Block a user