|
@@ -28,6 +28,7 @@ import static org.mockito.Mockito.mock;
|
|
|
import static org.mockito.Mockito.when;
|
|
|
import static org.mockito.Mockito.times;
|
|
|
|
|
|
+import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
import static org.junit.Assert.*;
|
|
@@ -148,6 +149,65 @@ public class TestFairCallQueue {
|
|
|
assertNull(fcq.poll());
|
|
|
}
|
|
|
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ @Test
|
|
|
+ public void testInsertionWithFailover() {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ // Config for server to throw StandbyException instead of the
|
|
|
+ // regular RetriableException if call queue is full.
|
|
|
+ conf.setBoolean(
|
|
|
+ "ns." + CommonConfigurationKeys.IPC_CALLQUEUE_SERVER_FAILOVER_ENABLE,
|
|
|
+ true);
|
|
|
+ // 3 queues, 2 slots each.
|
|
|
+ fcq = Mockito.spy(new FairCallQueue<>(3, 6, "ns", conf));
|
|
|
+
|
|
|
+ Schedulable p0 = mockCall("a", 0);
|
|
|
+ Schedulable p1 = mockCall("b", 1);
|
|
|
+
|
|
|
+ // add to first queue.
|
|
|
+ addToQueueAndVerify(p0, 1, 0, 0);
|
|
|
+ // 0:x- 1:-- 2:--
|
|
|
+
|
|
|
+ // add to second queue.
|
|
|
+ addToQueueAndVerify(p1, 0, 1, 0);
|
|
|
+ // 0:x- 1:x- 2:--
|
|
|
+
|
|
|
+ // add to first queue.
|
|
|
+ addToQueueAndVerify(p0, 1, 0, 0);
|
|
|
+ // 0:xx 1:x- 2:--
|
|
|
+
|
|
|
+ // add to first full queue spills over to second.
|
|
|
+ addToQueueAndVerify(p0, 1, 1, 0);
|
|
|
+ // 0:xx 1:xx 2:--
|
|
|
+
|
|
|
+ // add to second full queue spills over to third.
|
|
|
+ addToQueueAndVerify(p1, 0, 1, 1);
|
|
|
+ // 0:xx 1:xx 2:x-
|
|
|
+
|
|
|
+ // add to first and second full queue spills over to third.
|
|
|
+ addToQueueAndVerify(p0, 1, 1, 1);
|
|
|
+ // 0:xx 1:xx 2:xx
|
|
|
+
|
|
|
+ // adding non-lowest priority with all queues full throws a
|
|
|
+ // standby exception for client to try another server.
|
|
|
+ Mockito.reset(fcq);
|
|
|
+ try {
|
|
|
+ fcq.add(p0);
|
|
|
+ fail("didn't fail");
|
|
|
+ } catch (IllegalStateException ise) {
|
|
|
+ checkOverflowException(ise, RpcStatusProto.FATAL, true);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void addToQueueAndVerify(Schedulable call, int expectedQueue0,
|
|
|
+ int expectedQueue1, int expectedQueue2) {
|
|
|
+ Mockito.reset(fcq);
|
|
|
+ fcq.add(call);
|
|
|
+ Mockito.verify(fcq, times(expectedQueue0)).offerQueue(0, call);
|
|
|
+ Mockito.verify(fcq, times(expectedQueue1)).offerQueue(1, call);
|
|
|
+ Mockito.verify(fcq, times(expectedQueue2)).offerQueue(2, call);
|
|
|
+ }
|
|
|
+
|
|
|
@SuppressWarnings("unchecked") // for mock reset.
|
|
|
@Test
|
|
|
public void testInsertion() throws Exception {
|
|
@@ -215,7 +275,7 @@ public class TestFairCallQueue {
|
|
|
fcq.add(p0);
|
|
|
fail("didn't fail");
|
|
|
} catch (IllegalStateException ise) {
|
|
|
- checkOverflowException(ise, RpcStatusProto.ERROR);
|
|
|
+ checkOverflowException(ise, RpcStatusProto.ERROR, false);
|
|
|
}
|
|
|
Mockito.verify(fcq, times(1)).offerQueue(0, p0);
|
|
|
Mockito.verify(fcq, times(1)).offerQueue(1, p0);
|
|
@@ -228,7 +288,7 @@ public class TestFairCallQueue {
|
|
|
fcq.add(p1);
|
|
|
fail("didn't fail");
|
|
|
} catch (IllegalStateException ise) {
|
|
|
- checkOverflowException(ise, RpcStatusProto.ERROR);
|
|
|
+ checkOverflowException(ise, RpcStatusProto.ERROR, false);
|
|
|
}
|
|
|
Mockito.verify(fcq, times(0)).offerQueue(0, p1);
|
|
|
Mockito.verify(fcq, times(1)).offerQueue(1, p1);
|
|
@@ -241,7 +301,7 @@ public class TestFairCallQueue {
|
|
|
fcq.add(p2);
|
|
|
fail("didn't fail");
|
|
|
} catch (IllegalStateException ise) {
|
|
|
- checkOverflowException(ise, RpcStatusProto.FATAL);
|
|
|
+ checkOverflowException(ise, RpcStatusProto.FATAL, false);
|
|
|
}
|
|
|
Mockito.verify(fcq, times(0)).offerQueue(0, p2);
|
|
|
Mockito.verify(fcq, times(0)).offerQueue(1, p2);
|
|
@@ -280,7 +340,8 @@ public class TestFairCallQueue {
|
|
|
Mockito.verify(fcq, times(1)).putQueue(2, p2);
|
|
|
}
|
|
|
|
|
|
- private void checkOverflowException(Exception ex, RpcStatusProto status) {
|
|
|
+ private void checkOverflowException(Exception ex, RpcStatusProto status,
|
|
|
+ boolean failOverTriggered) {
|
|
|
// should be an overflow exception
|
|
|
assertTrue(ex.getClass().getName() + " != CallQueueOverflowException",
|
|
|
ex instanceof CallQueueOverflowException);
|
|
@@ -289,10 +350,15 @@ public class TestFairCallQueue {
|
|
|
assertTrue(ioe.getClass().getName() + " != RpcServerException",
|
|
|
ioe instanceof RpcServerException);
|
|
|
RpcServerException rse = (RpcServerException)ioe;
|
|
|
- // check error/fatal status and if it embeds a retriable ex.
|
|
|
+ // check error/fatal status and if it embeds a retriable ex or standby ex.
|
|
|
assertEquals(status, rse.getRpcStatusProto());
|
|
|
- assertTrue(rse.getClass().getName() + " != RetriableException",
|
|
|
- rse.getCause() instanceof RetriableException);
|
|
|
+ if (failOverTriggered) {
|
|
|
+ assertTrue(rse.getClass().getName() + " != RetriableException",
|
|
|
+ rse.getCause() instanceof StandbyException);
|
|
|
+ } else {
|
|
|
+ assertTrue(rse.getClass().getName() + " != RetriableException",
|
|
|
+ rse.getCause() instanceof RetriableException);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
//
|