@Test public void testShouldBlockWhenNotAtHead() throws InterruptedException { MockQueue q = new MockQueue(); final BlockingEnvelopeMap map = new MockBlockingEnvelopeMap(q); map.register(SSP, "0"); Thread t = new Thread(new Runnable() { @Override public void run() { try { // Should trigger a take() call. map.poll(FETCH, -1); } catch (InterruptedException e) { throw new RuntimeException(e); } } }); t.setDaemon(true); t.start(); q.awaitPollTimeout(); t.join(60000); // 1000 = blocking timeout constant assertEquals(1000, q.timeout); assertFalse(t.isAlive()); }
final BlockingEnvelopeMap map = new MockBlockingEnvelopeMap(q, new Clock() { private final long now = System.currentTimeMillis();
@Test public void testShouldBlockAtLeast100Ms() throws InterruptedException { BlockingEnvelopeMap map = new MockBlockingEnvelopeMap(); map.register(SSP, "0"); long now = System.currentTimeMillis(); map.poll(FETCH, 100); assertTrue(System.currentTimeMillis() - now >= 100); }
@Test public void testShouldGetSomeMessages() throws InterruptedException { BlockingEnvelopeMap map = new MockBlockingEnvelopeMap(); map.register(SSP, "0"); map.put(SSP, ENVELOPE); Map<SystemStreamPartition, List<IncomingMessageEnvelope>> envelopes = map.poll(FETCH, 0); assertEquals(1, envelopes.size()); assertEquals(1, envelopes.get(SSP).size()); map.put(SSP, ENVELOPE); map.put(SSP, ENVELOPE); envelopes = map.poll(FETCH, 0); assertEquals(1, envelopes.size()); assertEquals(2, envelopes.get(SSP).size()); // Size info. assertEquals(0, map.getMessagesSizeInQueue(SSP)); }
@Test public void testSizeComputation() throws InterruptedException { BlockingEnvelopeMap map = new MockBlockingEnvelopeMap(true); map.register(SSP, "0"); map.put(SSP, ENVELOPE_WITH_SIZE); map.put(SSP, ENVELOPE_WITH_SIZE); // Size info. assertEquals(200, map.getMessagesSizeInQueue(SSP)); Map<SystemStreamPartition, List<IncomingMessageEnvelope>> envelopes = map.poll(FETCH, 0); assertEquals(0, map.getMessagesSizeInQueue(SSP)); }
@Test public void testNoSizeComputation() throws InterruptedException { BlockingEnvelopeMap map = new MockBlockingEnvelopeMap(); map.register(SSP, "0"); map.put(SSP, ENVELOPE); map.put(SSP, ENVELOPE); Map<SystemStreamPartition, List<IncomingMessageEnvelope>> envelopes = map.poll(FETCH, 0); // Size info. assertEquals(0, map.getMessagesSizeInQueue(SSP)); }
@Test public void testEmptyMapReturnsEmptyList() throws InterruptedException { BlockingEnvelopeMap map = new MockBlockingEnvelopeMap(); map.register(SSP, "0"); map.poll(FETCH, 0); map.poll(FETCH, 30); map.setIsAtHead(SSP, true); map.poll(FETCH, -1); }