mirror of https://github.com/apache/druid.git
fix transient failures in unit tests
This commit is contained in:
parent
d94821998f
commit
38b0f1fbc2
|
@ -39,10 +39,8 @@ public class KafkaExtractionNamespace implements ExtractionNamespace
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public KafkaExtractionNamespace(
|
public KafkaExtractionNamespace(
|
||||||
@NotNull @JsonProperty(value = "kafkaTopic", required = true)
|
@NotNull @JsonProperty(value = "kafkaTopic", required = true) final String kafkaTopic,
|
||||||
final String kafkaTopic,
|
@NotNull @JsonProperty(value = "namespace", required = true) final String namespace
|
||||||
@NotNull @JsonProperty(value = "namespace", required = true)
|
|
||||||
final String namespace
|
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
Preconditions.checkNotNull(kafkaTopic, "kafkaTopic required");
|
Preconditions.checkNotNull(kafkaTopic, "kafkaTopic required");
|
||||||
|
|
|
@ -57,13 +57,12 @@ import org.apache.commons.io.FileUtils;
|
||||||
import org.apache.curator.test.TestingServer;
|
import org.apache.curator.test.TestingServer;
|
||||||
import org.apache.zookeeper.CreateMode;
|
import org.apache.zookeeper.CreateMode;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.junit.AfterClass;
|
import org.junit.After;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.BeforeClass;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -76,20 +75,20 @@ import java.util.concurrent.ConcurrentMap;
|
||||||
public class TestKafkaExtractionCluster
|
public class TestKafkaExtractionCluster
|
||||||
{
|
{
|
||||||
private static final Logger log = new Logger(TestKafkaExtractionCluster.class);
|
private static final Logger log = new Logger(TestKafkaExtractionCluster.class);
|
||||||
private static KafkaServer kafkaServer;
|
|
||||||
private static Properties kafkaProperties = new Properties();
|
|
||||||
private static KafkaConfig kafkaConfig;
|
|
||||||
private static final String topicName = "testTopic";
|
|
||||||
private static final String namespace = "testNamespace";
|
|
||||||
private static TestingServer zkTestServer;
|
|
||||||
private static KafkaExtractionManager renameManager;
|
|
||||||
|
|
||||||
private static final Lifecycle lifecycle = new Lifecycle();
|
private static final Lifecycle lifecycle = new Lifecycle();
|
||||||
private static NamespaceExtractionCacheManager extractionCacheManager;
|
private static final File tmpDir = Files.createTempDir();
|
||||||
private static ZkClient zkClient = null;
|
private static final String topicName = "testTopic";
|
||||||
private static File tmpDir = Files.createTempDir();
|
private static final String namespace = "testNamespace";
|
||||||
private static Injector injector;
|
private static final Properties kafkaProperties = new Properties();
|
||||||
|
|
||||||
|
private KafkaServer kafkaServer;
|
||||||
|
private KafkaConfig kafkaConfig;
|
||||||
|
private TestingServer zkTestServer;
|
||||||
|
private ZkClient zkClient;
|
||||||
|
private KafkaExtractionManager renameManager;
|
||||||
|
private NamespaceExtractionCacheManager extractionCacheManager;
|
||||||
|
private Injector injector;
|
||||||
|
|
||||||
public static class KafkaFactoryProvider implements Provider<ExtractionNamespaceFunctionFactory<?>>
|
public static class KafkaFactoryProvider implements Provider<ExtractionNamespaceFunctionFactory<?>>
|
||||||
{
|
{
|
||||||
|
@ -110,10 +109,12 @@ public class TestKafkaExtractionCluster
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@BeforeClass
|
@Before
|
||||||
public static void setupStatic() throws Exception
|
public void setUp() throws Exception
|
||||||
{
|
{
|
||||||
zkTestServer = new TestingServer(-1, new File(tmpDir.getAbsolutePath() + "/zk"), true);
|
zkTestServer = new TestingServer(-1, new File(tmpDir.getAbsolutePath() + "/zk"), true);
|
||||||
|
zkTestServer.start();
|
||||||
|
|
||||||
zkClient = new ZkClient(
|
zkClient = new ZkClient(
|
||||||
zkTestServer.getConnectString(),
|
zkTestServer.getConnectString(),
|
||||||
10000,
|
10000,
|
||||||
|
@ -142,38 +143,41 @@ public class TestKafkaExtractionCluster
|
||||||
|
|
||||||
final long time = DateTime.parse("2015-01-01").getMillis();
|
final long time = DateTime.parse("2015-01-01").getMillis();
|
||||||
kafkaServer = new KafkaServer(
|
kafkaServer = new KafkaServer(
|
||||||
kafkaConfig, new Time()
|
kafkaConfig,
|
||||||
{
|
new Time()
|
||||||
|
{
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long milliseconds()
|
public long milliseconds()
|
||||||
{
|
{
|
||||||
return time;
|
return time;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long nanoseconds()
|
public long nanoseconds()
|
||||||
{
|
{
|
||||||
return milliseconds() * 1_000_000;
|
return milliseconds() * 1_000_000;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void sleep(long ms)
|
public void sleep(long ms)
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
Thread.sleep(ms);
|
Thread.sleep(ms);
|
||||||
|
}
|
||||||
|
catch (InterruptedException e) {
|
||||||
|
throw Throwables.propagate(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
catch (InterruptedException e) {
|
|
||||||
throw Throwables.propagate(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
);
|
||||||
kafkaServer.startup();
|
kafkaServer.startup();
|
||||||
|
|
||||||
int sleepCount = 0;
|
int sleepCount = 0;
|
||||||
|
|
||||||
while (!kafkaServer.kafkaController().isActive()) {
|
while (!kafkaServer.kafkaController().isActive()) {
|
||||||
Thread.sleep(10);
|
Thread.sleep(100);
|
||||||
if (++sleepCount > 100) {
|
if (++sleepCount > 10) {
|
||||||
throw new InterruptedException("Controller took to long to awaken");
|
throw new InterruptedException("Controller took to long to awaken");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -184,6 +188,7 @@ public class TestKafkaExtractionCluster
|
||||||
zkTestServer.getConnectString() + "/kafka", 10000, 10000,
|
zkTestServer.getConnectString() + "/kafka", 10000, 10000,
|
||||||
ZKStringSerializer$.MODULE$
|
ZKStringSerializer$.MODULE$
|
||||||
);
|
);
|
||||||
|
|
||||||
try {
|
try {
|
||||||
final Properties topicProperties = new Properties();
|
final Properties topicProperties = new Properties();
|
||||||
topicProperties.put("cleanup.policy", "compact");
|
topicProperties.put("cleanup.policy", "compact");
|
||||||
|
@ -198,11 +203,13 @@ public class TestKafkaExtractionCluster
|
||||||
finally {
|
finally {
|
||||||
zkClient.close();
|
zkClient.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
final Properties kafkaProducerProperties = makeProducerProperties();
|
final Properties kafkaProducerProperties = makeProducerProperties();
|
||||||
Producer<byte[], byte[]> producer = new Producer<byte[], byte[]>(new ProducerConfig(kafkaProducerProperties));
|
Producer<byte[], byte[]> producer = new Producer<>(new ProducerConfig(kafkaProducerProperties));
|
||||||
|
|
||||||
try {
|
try {
|
||||||
producer.send(
|
producer.send(
|
||||||
new KeyedMessage<byte[], byte[]>(
|
new KeyedMessage<>(
|
||||||
topicName,
|
topicName,
|
||||||
StringUtils.toUtf8("abcdefg"),
|
StringUtils.toUtf8("abcdefg"),
|
||||||
StringUtils.toUtf8("abcdefg")
|
StringUtils.toUtf8("abcdefg")
|
||||||
|
@ -221,7 +228,8 @@ public class TestKafkaExtractionCluster
|
||||||
injector = Initialization.makeInjectorWithModules(
|
injector = Initialization.makeInjectorWithModules(
|
||||||
GuiceInjectors.makeStartupInjectorWithModules(
|
GuiceInjectors.makeStartupInjectorWithModules(
|
||||||
ImmutableList.<Module>of()
|
ImmutableList.<Module>of()
|
||||||
), ImmutableList.of(
|
),
|
||||||
|
ImmutableList.<Module>of(
|
||||||
new Module()
|
new Module()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
|
@ -230,7 +238,8 @@ public class TestKafkaExtractionCluster
|
||||||
binder.bindConstant().annotatedWith(Names.named("serviceName")).to("test");
|
binder.bindConstant().annotatedWith(Names.named("serviceName")).to("test");
|
||||||
binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0);
|
binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0);
|
||||||
}
|
}
|
||||||
}, new NamespacedExtractionModule(),
|
},
|
||||||
|
new NamespacedExtractionModule(),
|
||||||
new KafkaExtractionNamespaceModule()
|
new KafkaExtractionNamespaceModule()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
|
@ -255,9 +264,10 @@ public class TestKafkaExtractionCluster
|
||||||
extractionCacheManager.schedule(
|
extractionCacheManager.schedule(
|
||||||
new KafkaExtractionNamespace(topicName, namespace)
|
new KafkaExtractionNamespace(topicName, namespace)
|
||||||
);
|
);
|
||||||
|
|
||||||
long start = System.currentTimeMillis();
|
long start = System.currentTimeMillis();
|
||||||
while (renameManager.getBackgroundTaskCount() < 1) {
|
while (renameManager.getBackgroundTaskCount() < 1) {
|
||||||
Thread.sleep(10); // wait for map populator to start up
|
Thread.sleep(100); // wait for map populator to start up
|
||||||
if (System.currentTimeMillis() > start + 60_000) {
|
if (System.currentTimeMillis() > start + 60_000) {
|
||||||
throw new ISE("renameManager took too long to start");
|
throw new ISE("renameManager took too long to start");
|
||||||
}
|
}
|
||||||
|
@ -265,9 +275,10 @@ public class TestKafkaExtractionCluster
|
||||||
log.info("--------------------------- started rename manager ---------------------------");
|
log.info("--------------------------- started rename manager ---------------------------");
|
||||||
}
|
}
|
||||||
|
|
||||||
@AfterClass
|
@After
|
||||||
public static void closeStatic() throws IOException
|
public void tearDown() throws Exception
|
||||||
{
|
{
|
||||||
|
|
||||||
lifecycle.stop();
|
lifecycle.stop();
|
||||||
if (null != renameManager) {
|
if (null != renameManager) {
|
||||||
renameManager.stop();
|
renameManager.stop();
|
||||||
|
@ -297,7 +308,7 @@ public class TestKafkaExtractionCluster
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static final Properties makeProducerProperties()
|
private final Properties makeProducerProperties()
|
||||||
{
|
{
|
||||||
final Properties kafkaProducerProperties = new Properties();
|
final Properties kafkaProducerProperties = new Properties();
|
||||||
kafkaProducerProperties.putAll(kafkaProperties);
|
kafkaProducerProperties.putAll(kafkaProperties);
|
||||||
|
@ -309,55 +320,70 @@ public class TestKafkaExtractionCluster
|
||||||
return kafkaProducerProperties;
|
return kafkaProducerProperties;
|
||||||
}
|
}
|
||||||
|
|
||||||
private static void checkServer()
|
private void checkServer()
|
||||||
{
|
{
|
||||||
if (!kafkaServer.apis().controller().isActive()) {
|
if (!kafkaServer.apis().controller().isActive()) {
|
||||||
throw new ISE("server is not active!");
|
throw new ISE("server is not active!");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
//@Test(timeout = 5_000)
|
@Test(timeout = 60_000L)
|
||||||
@Test
|
|
||||||
public void testSimpleRename() throws InterruptedException
|
public void testSimpleRename() throws InterruptedException
|
||||||
{
|
{
|
||||||
final Properties kafkaProducerProperties = makeProducerProperties();
|
final Properties kafkaProducerProperties = makeProducerProperties();
|
||||||
final Producer<byte[], byte[]> producer = new Producer<byte[], byte[]>(new ProducerConfig(kafkaProducerProperties));
|
final Producer<byte[], byte[]> producer = new Producer<>(new ProducerConfig(kafkaProducerProperties));
|
||||||
|
|
||||||
try {
|
try {
|
||||||
checkServer();
|
checkServer();
|
||||||
final ConcurrentMap<String, Function<String, String>> fnFn = injector.getInstance(Key.get(new TypeLiteral<ConcurrentMap<String, Function<String, String>>>()
|
|
||||||
{
|
final ConcurrentMap<String, Function<String, String>> fnFn =
|
||||||
}, Names.named("namespaceExtractionFunctionCache")));
|
injector.getInstance(
|
||||||
final ConcurrentMap<String, Function<String, List<String>>> reverseFn = injector.getInstance(Key.get(new TypeLiteral<ConcurrentMap<String, Function<String, List<String>>>>()
|
Key.get(
|
||||||
{
|
new TypeLiteral<ConcurrentMap<String, Function<String, String>>>()
|
||||||
}, Names.named("namespaceReverseExtractionFunctionCache")));
|
{
|
||||||
|
},
|
||||||
|
Names.named("namespaceExtractionFunctionCache")
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
final ConcurrentMap<String, Function<String, List<String>>> reverseFn =
|
||||||
|
injector.getInstance(
|
||||||
|
Key.get(
|
||||||
|
new TypeLiteral<ConcurrentMap<String, Function<String, List<String>>>>()
|
||||||
|
{
|
||||||
|
},
|
||||||
|
Names.named("namespaceReverseExtractionFunctionCache")
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
KafkaExtractionNamespace extractionNamespace = new KafkaExtractionNamespace(topicName, namespace);
|
KafkaExtractionNamespace extractionNamespace = new KafkaExtractionNamespace(topicName, namespace);
|
||||||
|
|
||||||
Assert.assertEquals(null, fnFn.get(extractionNamespace.getNamespace()).apply("foo"));
|
assertUpdated(null, extractionNamespace.getNamespace(), "foo", fnFn);
|
||||||
Assert.assertEquals(Collections.EMPTY_LIST, reverseFn.get(extractionNamespace.getNamespace()).apply("foo"));
|
assertReverseUpdated(Collections.EMPTY_LIST, extractionNamespace.getNamespace(), "foo", reverseFn);
|
||||||
|
|
||||||
long events = renameManager.getNumEvents(namespace);
|
long events = renameManager.getNumEvents(namespace);
|
||||||
|
|
||||||
log.info("------------------------- Sending foo bar -------------------------------");
|
log.info("------------------------- Sending foo bar -------------------------------");
|
||||||
producer.send(new KeyedMessage<byte[], byte[]>(topicName, StringUtils.toUtf8("foo"), StringUtils.toUtf8("bar")));
|
producer.send(new KeyedMessage<>(topicName, StringUtils.toUtf8("foo"), StringUtils.toUtf8("bar")));
|
||||||
|
|
||||||
long start = System.currentTimeMillis();
|
long start = System.currentTimeMillis();
|
||||||
while (events == renameManager.getNumEvents(namespace)) {
|
while (events == renameManager.getNumEvents(namespace)) {
|
||||||
Thread.sleep(10);
|
Thread.sleep(100);
|
||||||
if (System.currentTimeMillis() > start + 60_000) {
|
if (System.currentTimeMillis() > start + 60_000) {
|
||||||
throw new ISE("Took too long to update event");
|
throw new ISE("Took too long to update event");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
log.info("------------------------- Checking foo bar -------------------------------");
|
log.info("------------------------- Checking foo bar -------------------------------");
|
||||||
Assert.assertEquals("bar", fnFn.get(extractionNamespace.getNamespace()).apply("foo"));
|
assertUpdated("bar", extractionNamespace.getNamespace(), "foo", fnFn);
|
||||||
Assert.assertEquals(Arrays.asList("foo"), reverseFn.get(extractionNamespace.getNamespace()).apply("bar"));
|
assertReverseUpdated(Arrays.asList("foo"), extractionNamespace.getNamespace(), "bar", reverseFn);
|
||||||
Assert.assertEquals(null, fnFn.get(extractionNamespace.getNamespace()).apply("baz"));
|
assertUpdated(null, extractionNamespace.getNamespace(), "baz", fnFn);
|
||||||
|
|
||||||
checkServer();
|
checkServer();
|
||||||
events = renameManager.getNumEvents(namespace);
|
events = renameManager.getNumEvents(namespace);
|
||||||
|
|
||||||
log.info("------------------------- Sending baz bat -------------------------------");
|
log.info("------------------------- Sending baz bat -------------------------------");
|
||||||
producer.send(new KeyedMessage<byte[], byte[]>(topicName, StringUtils.toUtf8("baz"), StringUtils.toUtf8("bat")));
|
producer.send(new KeyedMessage<>(topicName, StringUtils.toUtf8("baz"), StringUtils.toUtf8("bat")));
|
||||||
while (events == renameManager.getNumEvents(namespace)) {
|
while (events == renameManager.getNumEvents(namespace)) {
|
||||||
Thread.sleep(10);
|
Thread.sleep(10);
|
||||||
if (System.currentTimeMillis() > start + 60_000) {
|
if (System.currentTimeMillis() > start + 60_000) {
|
||||||
|
@ -373,4 +399,44 @@ public class TestKafkaExtractionCluster
|
||||||
producer.close();
|
producer.close();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void assertUpdated(
|
||||||
|
String expected,
|
||||||
|
String namespace,
|
||||||
|
String key,
|
||||||
|
ConcurrentMap<String, Function<String, String>> lookup
|
||||||
|
)
|
||||||
|
throws InterruptedException
|
||||||
|
{
|
||||||
|
final Function<String, String> extractionFn = lookup.get(namespace);
|
||||||
|
|
||||||
|
if (expected == null) {
|
||||||
|
while (extractionFn.apply(key) != null) {
|
||||||
|
Thread.sleep(100);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
while (!expected.equals(extractionFn.apply(key))) {
|
||||||
|
Thread.sleep(100);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
Assert.assertEquals("update check", expected, extractionFn.apply(key));
|
||||||
|
}
|
||||||
|
|
||||||
|
private void assertReverseUpdated(
|
||||||
|
List<String> expected,
|
||||||
|
String namespace,
|
||||||
|
String key,
|
||||||
|
ConcurrentMap<String, Function<String, List<String>>> lookup
|
||||||
|
)
|
||||||
|
throws InterruptedException
|
||||||
|
{
|
||||||
|
final Function<String, List<String>> extractionFn = lookup.get(namespace);
|
||||||
|
|
||||||
|
while (!extractionFn.apply(key).equals(expected)) {
|
||||||
|
Thread.sleep(100);
|
||||||
|
}
|
||||||
|
|
||||||
|
Assert.assertEquals("update check", expected, extractionFn.apply(key));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -36,7 +36,6 @@ import javax.validation.constraints.NotNull;
|
||||||
@JsonTypeName("jdbc")
|
@JsonTypeName("jdbc")
|
||||||
public class JDBCExtractionNamespace implements ExtractionNamespace
|
public class JDBCExtractionNamespace implements ExtractionNamespace
|
||||||
{
|
{
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
private final MetadataStorageConnectorConfig connectorConfig;
|
private final MetadataStorageConnectorConfig connectorConfig;
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
|
|
|
@ -413,7 +413,7 @@ public class JDBCExtractionNamespaceTest
|
||||||
assertUpdated(extractionNamespace.getNamespace(), "foo", "bar");
|
assertUpdated(extractionNamespace.getNamespace(), "foo", "bar");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout = 10_000L)
|
@Test(timeout = 60_000L)
|
||||||
public void testFindNew()
|
public void testFindNew()
|
||||||
throws NoSuchFieldException, IllegalAccessException, ExecutionException, InterruptedException
|
throws NoSuchFieldException, IllegalAccessException, ExecutionException, InterruptedException
|
||||||
{
|
{
|
||||||
|
@ -482,6 +482,12 @@ public class JDBCExtractionNamespaceTest
|
||||||
waitForUpdates(1_000L, 2L);
|
waitForUpdates(1_000L, 2L);
|
||||||
|
|
||||||
Function<String, String> extractionFn = fnCache.get(namespace);
|
Function<String, String> extractionFn = fnCache.get(namespace);
|
||||||
|
|
||||||
|
// rely on test timeout to break out of this loop
|
||||||
|
while (!extractionFn.apply(key).equals(expected)) {
|
||||||
|
Thread.sleep(100);
|
||||||
|
}
|
||||||
|
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
"update check",
|
"update check",
|
||||||
expected,
|
expected,
|
||||||
|
|
|
@ -168,7 +168,7 @@ public class RealtimeIndexTaskTest
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout = 10000L)
|
@Test(timeout = 60_000L)
|
||||||
public void testBasics() throws Exception
|
public void testBasics() throws Exception
|
||||||
{
|
{
|
||||||
final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator();
|
final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator();
|
||||||
|
@ -221,7 +221,7 @@ public class RealtimeIndexTaskTest
|
||||||
Assert.assertEquals(TaskStatus.Status.SUCCESS, taskStatus.getStatusCode());
|
Assert.assertEquals(TaskStatus.Status.SUCCESS, taskStatus.getStatusCode());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout = 10000L)
|
@Test(timeout = 60_000L)
|
||||||
public void testRestore() throws Exception
|
public void testRestore() throws Exception
|
||||||
{
|
{
|
||||||
final File directory = tempFolder.newFolder();
|
final File directory = tempFolder.newFolder();
|
||||||
|
|
|
@ -907,7 +907,7 @@ public class TaskLifecycleTest
|
||||||
EasyMock.verify(monitorScheduler, queryRunnerFactoryConglomerate);
|
EasyMock.verify(monitorScheduler, queryRunnerFactoryConglomerate);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout = 4000L)
|
@Test(timeout = 60_000L)
|
||||||
public void testRealtimeIndexTaskFailure() throws Exception
|
public void testRealtimeIndexTaskFailure() throws Exception
|
||||||
{
|
{
|
||||||
setUpAndStartTaskQueue(
|
setUpAndStartTaskQueue(
|
||||||
|
|
|
@ -202,7 +202,7 @@ public class SimpleResourceManagementStrategyTest
|
||||||
ServiceEmitter emitter = EasyMock.createMock(ServiceEmitter.class);
|
ServiceEmitter emitter = EasyMock.createMock(ServiceEmitter.class);
|
||||||
EmittingLogger.registerEmitter(emitter);
|
EmittingLogger.registerEmitter(emitter);
|
||||||
emitter.emit(EasyMock.<ServiceEventBuilder>anyObject());
|
emitter.emit(EasyMock.<ServiceEventBuilder>anyObject());
|
||||||
EasyMock.expectLastCall();
|
EasyMock.expectLastCall().atLeastOnce();
|
||||||
EasyMock.replay(emitter);
|
EasyMock.replay(emitter);
|
||||||
|
|
||||||
EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0).times(2);
|
EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0).times(2);
|
||||||
|
|
|
@ -20,6 +20,7 @@
|
||||||
package io.druid.curator.announcement;
|
package io.druid.curator.announcement;
|
||||||
|
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
|
import com.metamx.common.ISE;
|
||||||
import io.druid.concurrent.Execs;
|
import io.druid.concurrent.Execs;
|
||||||
import io.druid.curator.CuratorTestBase;
|
import io.druid.curator.CuratorTestBase;
|
||||||
import org.apache.curator.framework.CuratorFramework;
|
import org.apache.curator.framework.CuratorFramework;
|
||||||
|
@ -172,6 +173,14 @@ public class AnnouncerTest extends CuratorTestBase
|
||||||
|
|
||||||
announcer.stop();
|
announcer.stop();
|
||||||
|
|
||||||
|
int count = 0;
|
||||||
|
while ((curator.checkExists().forPath(testPath1) != null) || (curator.checkExists().forPath(testPath1) != null)) {
|
||||||
|
Thread.sleep(100);
|
||||||
|
if (++count > 10) {
|
||||||
|
throw new ISE("Curator paths not getting cleaned up");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
Assert.assertNull(curator.checkExists().forPath(testPath1));
|
Assert.assertNull(curator.checkExists().forPath(testPath1));
|
||||||
Assert.assertNull(curator.checkExists().forPath(testPath2));
|
Assert.assertNull(curator.checkExists().forPath(testPath2));
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue