Skip to content

Commit

Permalink
Adding tests and addressing comments
Browse files Browse the repository at this point in the history
  • Loading branch information
pranavbhole committed Aug 24, 2023
1 parent c2a04cc commit dfa7e3b
Show file tree
Hide file tree
Showing 17 changed files with 202 additions and 32 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -298,13 +298,13 @@ public LookupIntrospectHandler getIntrospectHandler()
}

@Override
public void awaitToInitialise() throws InterruptedException, TimeoutException
public void awaitInitialization()
{

}

@Override
public boolean isCacheLoaded()
public boolean isInitialized()
{
return true;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -83,11 +83,13 @@ public void testSimpleSerDe() throws Exception
mapper.writeValueAsString(expected),
KafkaLookupExtractorFactory.class
);
result.awaitInitialization();
Assert.assertEquals(expected.getKafkaTopic(), result.getKafkaTopic());
Assert.assertEquals(expected.getKafkaProperties(), result.getKafkaProperties());
Assert.assertEquals(cacheManager, result.getCacheManager());
Assert.assertEquals(0, expected.getCompletedEventCount());
Assert.assertEquals(0, result.getCompletedEventCount());
Assert.assertTrue(result.isInitialized());
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -171,7 +171,7 @@ public LookupIntrospectHandler getIntrospectHandler()
}

@Override
public void awaitToInitialise() throws InterruptedException, TimeoutException
public void awaitInitialization() throws InterruptedException, TimeoutException
{
long timeout = extractionNamespace.getLoadTimeout();
if (entry.getCacheState() == CacheScheduler.NoCache.CACHE_NOT_INITIALIZED) {
Expand All @@ -181,7 +181,7 @@ public void awaitToInitialise() throws InterruptedException, TimeoutException
}

@Override
public boolean isCacheLoaded()
public boolean isInitialized()
{
return entry.getCacheState() instanceof CacheScheduler.VersionedCache;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,7 @@
import static org.mockito.Mockito.atMostOnce;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.reset;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoInteractions;
import static org.mockito.Mockito.verifyNoMoreInteractions;
Expand Down Expand Up @@ -227,7 +228,6 @@ public void testSimpleStartStart() throws Exception
);
Assert.assertTrue(namespaceLookupExtractorFactory.start());
Assert.assertTrue(namespaceLookupExtractorFactory.start());

verify(scheduler).scheduleAndWait(extractionNamespace, 60000L);
verifyNoMoreInteractions(scheduler, entry, versionedCache);
}
Expand Down Expand Up @@ -287,6 +287,40 @@ public long getPollMs()
verifyNoMoreInteractions(scheduler, entry, versionedCache);
}

@Test
public void testAwaitInitializationOnCacheNotInitialized() throws Exception
{
final ExtractionNamespace extractionNamespace = new ExtractionNamespace()
{
@Override
public long getPollMs()
{
return 0;
}

@Override
public long getLoadTimeout()
{
return 1;
}
};
expectScheduleAndWaitOnce(extractionNamespace);
when(entry.getCacheState()).thenReturn(CacheScheduler.NoCache.CACHE_NOT_INITIALIZED);

final NamespaceLookupExtractorFactory namespaceLookupExtractorFactory = new NamespaceLookupExtractorFactory(
extractionNamespace,
scheduler
);
Assert.assertTrue(namespaceLookupExtractorFactory.start());
namespaceLookupExtractorFactory.awaitInitialization();
Assert.assertThrows(ISE.class, () -> namespaceLookupExtractorFactory.get());
verify(scheduler).scheduleAndWait(extractionNamespace, 60000L);
verify(entry, times(2)).getCacheState();
verify(entry).awaitTotalUpdatesWithTimeout(1, 1);
Thread.sleep(10);
verifyNoMoreInteractions(scheduler, entry, versionedCache);
}

private void expectScheduleAndWaitOnce(ExtractionNamespace extractionNamespace)
{
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -191,6 +191,24 @@ public void testSimpleSubmission() throws InterruptedException
Assert.assertEquals(VALUE, entry.getCache().get(KEY));
}

@Test(timeout = 60_000L)
public void testInitialization() throws InterruptedException, TimeoutException
{
UriExtractionNamespace namespace = new UriExtractionNamespace(
tmpFile.toURI(),
null, null,
new UriExtractionNamespace.ObjectMapperFlatDataParser(
UriExtractionNamespaceTest.registerTypes(new ObjectMapper())
),
new Period(0),
null,
null
);
CacheScheduler.Entry entry = scheduler.schedule(namespace);
entry.awaitTotalUpdatesWithTimeout(1, 2000);
Assert.assertEquals(VALUE, entry.getCache().get(KEY));
}

@Test(timeout = 60_000L)
public void testPeriodicUpdatesScheduled() throws InterruptedException
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@
import javax.annotation.Nullable;
import java.util.List;
import java.util.Objects;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicBoolean;

@JsonTypeName("loadingLookup")
Expand Down Expand Up @@ -113,13 +112,13 @@ public LookupIntrospectHandler getIntrospectHandler()
}

@Override
public void awaitToInitialise() throws InterruptedException, TimeoutException
public void awaitInitialization()
{

}

@Override
public boolean isCacheLoaded()
public boolean isInitialized()
{
return true;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -129,12 +129,12 @@ public LookupIntrospectHandler getIntrospectHandler()
}

@Override
public void awaitToInitialise()
public void awaitInitialization()
{
}

@Override
public boolean isCacheLoaded()
public boolean isInitialized()
{
return true;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,8 @@ public void testStartStop()
EasyMock.expectLastCall().once();
EasyMock.replay(loadingLookup);
Assert.assertTrue(loadingLookupFactory.start());
loadingLookupFactory.awaitInitialization();
Assert.assertTrue(loadingLookupFactory.isInitialized());
Assert.assertTrue(loadingLookupFactory.close());
EasyMock.verify(loadingLookup);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,8 @@ public void testStart()
EasyMock.expect(pollingLookup.isOpen()).andReturn(true).once();
EasyMock.replay(pollingLookup);
Assert.assertTrue(pollingLookupFactory.start());
pollingLookupFactory.awaitInitialization();
Assert.assertTrue(pollingLookupFactory.isInitialized());
EasyMock.verify(pollingLookup);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -84,10 +84,10 @@ default boolean destroy()
/**
* awaitToInitialise blocks and wait for the cache to initialize fully.
*/
void awaitToInitialise() throws InterruptedException, TimeoutException;
void awaitInitialization() throws InterruptedException, TimeoutException;

/**
* @return true if cache is loaded and lookup is queryable else returns false
*/
boolean isCacheLoaded();
boolean isInitialized();
}
Original file line number Diff line number Diff line change
Expand Up @@ -105,12 +105,12 @@ public LookupIntrospectHandler getIntrospectHandler()
}

@Override
public void awaitToInitialise()
public void awaitInitialization()
{
}

@Override
public boolean isCacheLoaded()
public boolean isInitialized()
{
return true;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -83,12 +83,12 @@ public LookupIntrospectHandler getIntrospectHandler()
}

@Override
public void awaitToInitialise()
public void awaitInitialization()
{
}

@Override
public boolean isCacheLoaded()
public boolean isInitialized()
{
return true;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -664,7 +664,7 @@ public void handle(Map<String, LookupExtractorFactoryContainer> lookupMap, Looku
e -> true,
startRetries
);
if (lookupExtractorFactoryContainer.getLookupExtractorFactory().isCacheLoaded()) {
if (lookupExtractorFactoryContainer.getLookupExtractorFactory().isInitialized()) {
old = lookupMap.put(lookupName, lookupExtractorFactoryContainer);
LOG.debug("Loaded lookup [%s] with spec [%s].", lookupName, lookupExtractorFactoryContainer);
manager.dropContainer(old, lookupName);
Expand All @@ -679,12 +679,12 @@ public void handle(Map<String, LookupExtractorFactoryContainer> lookupMap, Looku
*/
RetryUtils.retry(
() -> {
lookupExtractorFactoryContainer.getLookupExtractorFactory().awaitToInitialise();
lookupExtractorFactoryContainer.getLookupExtractorFactory().awaitInitialization();
return null;
}, e -> true,
startRetries
);
if (lookupExtractorFactoryContainer.getLookupExtractorFactory().isCacheLoaded()) {
if (lookupExtractorFactoryContainer.getLookupExtractorFactory().isInitialized()) {
// send load notice with cache loaded container
manager.add(lookupName, lookupExtractorFactoryContainer);
} else {
Expand Down Expand Up @@ -731,11 +731,11 @@ private static class DropNotice implements Notice
@Override
public void handle(Map<String, LookupExtractorFactoryContainer> lookupMap, LookupReferencesManager manager)
{
if (loadedContainer != null && !loadedContainer.getLookupExtractorFactory().isCacheLoaded()) {
if (loadedContainer != null && !loadedContainer.getLookupExtractorFactory().isInitialized()) {
final LookupExtractorFactoryContainer containterToDrop = lookupMap.get(lookupName);
manager.submitAsyncLookupTask(() -> {
try {
loadedContainer.getLookupExtractorFactory().awaitToInitialise();
loadedContainer.getLookupExtractorFactory().awaitInitialization();
manager.dropContainer(containterToDrop, lookupName);
}
catch (InterruptedException | TimeoutException e) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,6 @@
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import java.util.Map;
import java.util.concurrent.TimeoutException;

@JsonTypeName("map")
public class MapLookupExtractorFactory implements LookupExtractorFactory
Expand Down Expand Up @@ -89,13 +88,13 @@ public LookupIntrospectHandler getIntrospectHandler()
}

@Override
public void awaitToInitialise() throws InterruptedException, TimeoutException
public void awaitInitialization()
{

}

@Override
public boolean isCacheLoaded()
public boolean isInitialized()
{
return true;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -99,12 +99,12 @@ public LookupIntrospectHandler getIntrospectHandler()
}

@Override
public void awaitToInitialise()
public void awaitInitialization()
{
}

@Override
public boolean isCacheLoaded()
public boolean isInitialized()
{
return true;
}
Expand Down
Loading

0 comments on commit dfa7e3b

Please sign in to comment.