From 05151bc3250d16ede6873c985ab0c678c13866bd Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 25 Mar 2016 20:33:53 -0700 Subject: [PATCH] Fix LookupCoordinatorManagerTest for alerts * Also fixes bad alerting on missing nodes --- .../cache/LookupCoordinatorManager.java | 2 +- .../cache/LookupCoordinatorManagerTest.java | 39 +++++++++++++++++-- 2 files changed, 36 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java b/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java index 05617fef62a..1258975f35c 100644 --- a/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java +++ b/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java @@ -159,7 +159,7 @@ public class LookupCoordinatorManager lookupCoordinatorManagerConfig.getHostDeleteTimeout() ).get()) { // 404 is ok here, that means it was already deleted - if (!httpStatusIsSuccess(returnCode.get()) || !httpStatusIsNotFound(returnCode.get())) { + if (!httpStatusIsSuccess(returnCode.get()) && !httpStatusIsNotFound(returnCode.get())) { final ByteArrayOutputStream baos = new ByteArrayOutputStream(); try { StreamUtils.copyAndClose(result, baos); diff --git a/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java b/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java index cbf9512b0b4..412e6a6b193 100644 --- a/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java +++ b/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java @@ -28,6 +28,7 @@ import com.google.common.net.HostAndPort; import com.google.common.util.concurrent.SettableFuture; import com.metamx.common.ISE; import com.metamx.common.StringUtils; +import com.metamx.emitter.core.Event; import com.metamx.emitter.core.LoggingEmitter; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.http.client.HttpClient; @@ -40,13 +41,13 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.query.lookup.LookupModule; import io.druid.server.listener.announcer.ListenerDiscoverer; import io.druid.server.listener.resource.ListenerResource; -import io.druid.server.lookup.cache.LookupCoordinatorManager; -import io.druid.server.lookup.cache.LookupCoordinatorManagerConfig; import org.easymock.EasyMock; import org.hamcrest.BaseMatcher; import org.hamcrest.Description; import org.joda.time.Duration; +import org.junit.After; import org.junit.Assert; +import org.junit.Before; import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; @@ -62,6 +63,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; public class LookupCoordinatorManagerTest @@ -89,12 +91,38 @@ public class LookupCoordinatorManagerTest SINGLE_LOOKUP_MAP ); private static final Map>> EMPTY_TIERED_LOOKUP = (Map>>) ImmutableMap.>>of(); + private static final AtomicLong EVENT_EMITS = new AtomicLong(0L); + private static ServiceEmitter SERVICE_EMITTER; @BeforeClass public static void setUpStatic() { - final LoggingEmitter loggingEmitter = EasyMock.createNiceMock(LoggingEmitter.class); - com.metamx.emitter.EmittingLogger.registerEmitter(new ServiceEmitter("", "", loggingEmitter)); + LoggingEmitter loggingEmitter = EasyMock.createNiceMock(LoggingEmitter.class); + EasyMock.replay(loggingEmitter); + SERVICE_EMITTER = new ServiceEmitter("", "", loggingEmitter) + { + @Override + public void emit(Event event) + { + EVENT_EMITS.incrementAndGet(); + super.emit(event); + } + }; + com.metamx.emitter.EmittingLogger.registerEmitter(SERVICE_EMITTER); + } + + @Before + public void setUp() throws IOException + { + SERVICE_EMITTER.flush(); + EVENT_EMITS.set(0L); + } + + @After + public void tearDown() throws IOException + { + SERVICE_EMITTER.flush(); + Assert.assertEquals(0, EVENT_EMITS.get()); } @Test @@ -431,6 +459,9 @@ public class LookupCoordinatorManagerTest }; // Should log and pass io exception manager.updateAllOnTier(LOOKUP_TIER, SINGLE_LOOKUP_MAP); + SERVICE_EMITTER.flush(); + Assert.assertEquals(1, EVENT_EMITS.get()); + EVENT_EMITS.set(0); } @Test