mirror of
https://github.com/apache/druid.git
synced 2025-02-17 07:25:02 +00:00
add supervisor reset API (#3484)
* add supervisor reset API * CR doc changes and kill running tasks / clear offsets from supervisor
This commit is contained in:
parent
6099d20303
commit
ca9114b41b
@ -222,6 +222,24 @@ GET /druid/indexer/v1/supervisor/<supervisorId>/history
|
|||||||
```
|
```
|
||||||
Returns an audit history of specs for the supervisor with the provided ID.
|
Returns an audit history of specs for the supervisor with the provided ID.
|
||||||
|
|
||||||
|
#### Reset Supervisor
|
||||||
|
```
|
||||||
|
POST /druid/indexer/v1/supervisor/<supervisorId>/reset
|
||||||
|
```
|
||||||
|
The indexing service keeps track of the latest persisted Kafka offsets in order to provide exactly-once ingestion
|
||||||
|
guarantees across tasks. Subsequent tasks must start reading from where the previous task completed in order for the
|
||||||
|
generated segments to be accepted. If the messages at the expected starting offsets are no longer available in Kafka
|
||||||
|
(typically because the message retention period has elapsed or the topic was removed and re-created) the supervisor will
|
||||||
|
refuse to start and in-flight tasks will fail.
|
||||||
|
|
||||||
|
This endpoint can be used to clear the stored offsets which will cause the supervisor to start reading from
|
||||||
|
either the earliest or latest offsets in Kafka (depending on the value of `useEarliestOffset`). The supervisor must be
|
||||||
|
running for this endpoint to be available. After the stored offsets are cleared, the supervisor will automatically kill
|
||||||
|
and re-create any active tasks so that tasks begin reading from valid offsets.
|
||||||
|
|
||||||
|
Note that since the stored offsets are necessary to guarantee exactly-once ingestion, resetting them with this endpoint
|
||||||
|
may cause some Kafka messages to be skipped or to be read twice.
|
||||||
|
|
||||||
## Capacity Planning
|
## Capacity Planning
|
||||||
|
|
||||||
Kafka indexing tasks run on middle managers and are thus limited by the resources available in the middle manager
|
Kafka indexing tasks run on middle managers and are thus limited by the resources available in the middle manager
|
||||||
|
@ -398,6 +398,12 @@ public class KafkaSupervisor implements Supervisor
|
|||||||
return generateReport(true);
|
return generateReport(true);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void reset() {
|
||||||
|
log.info("Posting ResetNotice");
|
||||||
|
notices.add(new ResetNotice());
|
||||||
|
}
|
||||||
|
|
||||||
public void possiblyRegisterListener()
|
public void possiblyRegisterListener()
|
||||||
{
|
{
|
||||||
// getTaskRunner() sometimes fails if the task queue is still being initialized so retry later until we succeed
|
// getTaskRunner() sometimes fails if the task queue is still being initialized so retry later until we succeed
|
||||||
@ -479,6 +485,33 @@ public class KafkaSupervisor implements Supervisor
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private class ResetNotice implements Notice
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public void handle()
|
||||||
|
{
|
||||||
|
resetInternal();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
void resetInternal()
|
||||||
|
{
|
||||||
|
boolean result = indexerMetadataStorageCoordinator.deleteDataSourceMetadata(dataSource);
|
||||||
|
log.info("Reset dataSource[%s] - dataSource metadata entry deleted? [%s]", dataSource, result);
|
||||||
|
|
||||||
|
for (TaskGroup taskGroup : taskGroups.values()) {
|
||||||
|
for (Map.Entry<String, TaskData> entry : taskGroup.tasks.entrySet()) {
|
||||||
|
String taskId = entry.getKey();
|
||||||
|
log.info("Reset dataSource[%s] - killing task [%s]", dataSource, taskId);
|
||||||
|
killTask(taskId);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
partitionGroups.clear();
|
||||||
|
taskGroups.clear();
|
||||||
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
void gracefulShutdownInternal() throws ExecutionException, InterruptedException
|
void gracefulShutdownInternal() throws ExecutionException, InterruptedException
|
||||||
{
|
{
|
||||||
@ -1289,8 +1322,9 @@ public class KafkaSupervisor implements Supervisor
|
|||||||
long latestKafkaOffset = getOffsetFromKafkaForPartition(partition, false);
|
long latestKafkaOffset = getOffsetFromKafkaForPartition(partition, false);
|
||||||
if (offset > latestKafkaOffset) {
|
if (offset > latestKafkaOffset) {
|
||||||
throw new ISE(
|
throw new ISE(
|
||||||
"Offset in metadata storage [%,d] > latest Kafka offset [%,d] for partition [%d]. If your Kafka offsets have"
|
"Offset in metadata storage [%,d] > latest Kafka offset [%,d] for partition[%d] dataSource[%s]. If these "
|
||||||
+ " been reset, you will need to remove the entry for [%s] from the dataSource table.",
|
+ "messages are no longer available (perhaps you deleted and re-created your Kafka topic) you can use the "
|
||||||
|
+ "supervisor reset API to restart ingestion.",
|
||||||
offset,
|
offset,
|
||||||
latestKafkaOffset,
|
latestKafkaOffset,
|
||||||
partition,
|
partition,
|
||||||
|
@ -1459,6 +1459,110 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
|||||||
verifyAll();
|
verifyAll();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testResetNoTasks() throws Exception
|
||||||
|
{
|
||||||
|
supervisor = getSupervisor(1, 1, true, "PT1H", null);
|
||||||
|
expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
|
||||||
|
expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
|
||||||
|
expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes();
|
||||||
|
expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.<Task>of()).anyTimes();
|
||||||
|
taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class));
|
||||||
|
replayAll();
|
||||||
|
|
||||||
|
supervisor.start();
|
||||||
|
supervisor.runInternal();
|
||||||
|
verifyAll();
|
||||||
|
|
||||||
|
reset(indexerMetadataStorageCoordinator);
|
||||||
|
expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true);
|
||||||
|
replay(indexerMetadataStorageCoordinator);
|
||||||
|
|
||||||
|
supervisor.resetInternal();
|
||||||
|
verifyAll();
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testResetRunningTasks() throws Exception
|
||||||
|
{
|
||||||
|
final TaskLocation location1 = new TaskLocation("testHost", 1234);
|
||||||
|
final TaskLocation location2 = new TaskLocation("testHost2", 145);
|
||||||
|
final DateTime startTime = new DateTime();
|
||||||
|
|
||||||
|
supervisor = getSupervisor(2, 1, true, "PT1H", null);
|
||||||
|
addSomeEvents(1);
|
||||||
|
|
||||||
|
Task id1 = createKafkaIndexTask(
|
||||||
|
"id1",
|
||||||
|
DATASOURCE,
|
||||||
|
"sequenceName-0",
|
||||||
|
new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
|
||||||
|
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
||||||
|
null
|
||||||
|
);
|
||||||
|
|
||||||
|
Task id2 = createKafkaIndexTask(
|
||||||
|
"id2",
|
||||||
|
DATASOURCE,
|
||||||
|
"sequenceName-0",
|
||||||
|
new KafkaPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
|
||||||
|
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
||||||
|
null
|
||||||
|
);
|
||||||
|
|
||||||
|
Task id3 = createKafkaIndexTask(
|
||||||
|
"id3",
|
||||||
|
DATASOURCE,
|
||||||
|
"sequenceName-0",
|
||||||
|
new KafkaPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
|
||||||
|
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
||||||
|
null
|
||||||
|
);
|
||||||
|
|
||||||
|
Collection workItems = new ArrayList<>();
|
||||||
|
workItems.add(new TestTaskRunnerWorkItem(id1.getId(), null, location1));
|
||||||
|
workItems.add(new TestTaskRunnerWorkItem(id2.getId(), null, location2));
|
||||||
|
|
||||||
|
expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
|
||||||
|
expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
|
||||||
|
expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes();
|
||||||
|
expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes();
|
||||||
|
expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes();
|
||||||
|
expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes();
|
||||||
|
expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes();
|
||||||
|
expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes();
|
||||||
|
expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes();
|
||||||
|
expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes();
|
||||||
|
expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
|
||||||
|
new KafkaDataSourceMetadata(
|
||||||
|
null
|
||||||
|
)
|
||||||
|
).anyTimes();
|
||||||
|
expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING));
|
||||||
|
expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING));
|
||||||
|
expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING));
|
||||||
|
expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime));
|
||||||
|
expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime));
|
||||||
|
expect(taskClient.getCurrentOffsets("id1", true)).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L));
|
||||||
|
|
||||||
|
taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class));
|
||||||
|
replayAll();
|
||||||
|
|
||||||
|
supervisor.start();
|
||||||
|
supervisor.runInternal();
|
||||||
|
verifyAll();
|
||||||
|
|
||||||
|
reset(taskQueue, indexerMetadataStorageCoordinator);
|
||||||
|
expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true);
|
||||||
|
taskQueue.shutdown("id2");
|
||||||
|
taskQueue.shutdown("id3");
|
||||||
|
replay(taskQueue, indexerMetadataStorageCoordinator);
|
||||||
|
|
||||||
|
supervisor.resetInternal();
|
||||||
|
verifyAll();
|
||||||
|
}
|
||||||
|
|
||||||
private void addSomeEvents(int numEventsPerPartition) throws Exception
|
private void addSomeEvents(int numEventsPerPartition) throws Exception
|
||||||
{
|
{
|
||||||
try (final KafkaProducer<byte[], byte[]> kafkaProducer = kafkaServer.newProducer()) {
|
try (final KafkaProducer<byte[], byte[]> kafkaProducer = kafkaServer.newProducer()) {
|
||||||
|
@ -138,6 +138,21 @@ public class SupervisorManager
|
|||||||
return supervisor == null ? Optional.<SupervisorReport>absent() : Optional.fromNullable(supervisor.lhs.getStatus());
|
return supervisor == null ? Optional.<SupervisorReport>absent() : Optional.fromNullable(supervisor.lhs.getStatus());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public boolean resetSupervisor(String id)
|
||||||
|
{
|
||||||
|
Preconditions.checkState(started, "SupervisorManager not started");
|
||||||
|
Preconditions.checkNotNull(id, "id");
|
||||||
|
|
||||||
|
Pair<Supervisor, SupervisorSpec> supervisor = supervisors.get(id);
|
||||||
|
|
||||||
|
if (supervisor == null) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
supervisor.lhs.reset();
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Stops a supervisor with a given id and then removes it from the list.
|
* Stops a supervisor with a given id and then removes it from the list.
|
||||||
* <p>
|
* <p>
|
||||||
|
@ -206,6 +206,29 @@ public class SupervisorResource
|
|||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@POST
|
||||||
|
@Path("/{id}/reset")
|
||||||
|
@Produces(MediaType.APPLICATION_JSON)
|
||||||
|
public Response reset(@PathParam("id") final String id)
|
||||||
|
{
|
||||||
|
return asLeaderWithSupervisorManager(
|
||||||
|
new Function<SupervisorManager, Response>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Response apply(SupervisorManager manager)
|
||||||
|
{
|
||||||
|
if (manager.resetSupervisor(id)) {
|
||||||
|
return Response.ok(ImmutableMap.of("id", id)).build();
|
||||||
|
} else {
|
||||||
|
return Response.status(Response.Status.NOT_FOUND)
|
||||||
|
.entity(ImmutableMap.of("error", String.format("[%s] does not exist", id)))
|
||||||
|
.build();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
private Response asLeaderWithSupervisorManager(Function<SupervisorManager, Response> f)
|
private Response asLeaderWithSupervisorManager(Function<SupervisorManager, Response> f)
|
||||||
{
|
{
|
||||||
Optional<SupervisorManager> supervisorManager = taskMaster.getSupervisorManager();
|
Optional<SupervisorManager> supervisorManager = taskMaster.getSupervisorManager();
|
||||||
|
@ -249,6 +249,25 @@ public class SupervisorManagerTest extends EasyMockSupport
|
|||||||
verifyAll();
|
verifyAll();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testResetSupervisor() throws Exception
|
||||||
|
{
|
||||||
|
Map<String, SupervisorSpec> existingSpecs = ImmutableMap.<String, SupervisorSpec>of(
|
||||||
|
"id1", new TestSupervisorSpec("id1", supervisor1)
|
||||||
|
);
|
||||||
|
|
||||||
|
EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs);
|
||||||
|
supervisor1.start();
|
||||||
|
supervisor1.reset();
|
||||||
|
replayAll();
|
||||||
|
|
||||||
|
manager.start();
|
||||||
|
Assert.assertTrue("resetValidSupervisor", manager.resetSupervisor("id1"));
|
||||||
|
Assert.assertFalse("resetInvalidSupervisor", manager.resetSupervisor("nobody_home"));
|
||||||
|
|
||||||
|
verifyAll();
|
||||||
|
}
|
||||||
|
|
||||||
private class TestSupervisorSpec implements SupervisorSpec
|
private class TestSupervisorSpec implements SupervisorSpec
|
||||||
{
|
{
|
||||||
private final String id;
|
private final String id;
|
||||||
|
@ -267,6 +267,35 @@ public class SupervisorResourceTest extends EasyMockSupport
|
|||||||
Assert.assertEquals(503, response.getStatus());
|
Assert.assertEquals(503, response.getStatus());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testReset() throws Exception
|
||||||
|
{
|
||||||
|
EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).times(2);
|
||||||
|
EasyMock.expect(supervisorManager.resetSupervisor("my-id")).andReturn(true);
|
||||||
|
EasyMock.expect(supervisorManager.resetSupervisor("my-id-2")).andReturn(false);
|
||||||
|
replayAll();
|
||||||
|
|
||||||
|
Response response = supervisorResource.reset("my-id");
|
||||||
|
|
||||||
|
Assert.assertEquals(200, response.getStatus());
|
||||||
|
Assert.assertEquals(ImmutableMap.of("id", "my-id"), response.getEntity());
|
||||||
|
|
||||||
|
response = supervisorResource.reset("my-id-2");
|
||||||
|
|
||||||
|
Assert.assertEquals(404, response.getStatus());
|
||||||
|
verifyAll();
|
||||||
|
|
||||||
|
resetAll();
|
||||||
|
|
||||||
|
EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.<SupervisorManager>absent());
|
||||||
|
replayAll();
|
||||||
|
|
||||||
|
response = supervisorResource.shutdown("my-id");
|
||||||
|
|
||||||
|
Assert.assertEquals(503, response.getStatus());
|
||||||
|
verifyAll();
|
||||||
|
}
|
||||||
|
|
||||||
private class TestSupervisorSpec implements SupervisorSpec
|
private class TestSupervisorSpec implements SupervisorSpec
|
||||||
{
|
{
|
||||||
private final String id;
|
private final String id;
|
||||||
|
@ -51,6 +51,12 @@ public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataSto
|
|||||||
throw new UnsupportedOperationException();
|
throw new UnsupportedOperationException();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean deleteDataSourceMetadata(String dataSource)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<DataSegment> getUsedSegmentsForInterval(String dataSource, Interval interval) throws IOException
|
public List<DataSegment> getUsedSegmentsForInterval(String dataSource, Interval interval) throws IOException
|
||||||
{
|
{
|
||||||
|
@ -54,7 +54,7 @@ public interface IndexerMetadataStorageCoordinator
|
|||||||
*
|
*
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public List<DataSegment> getUsedSegmentsForIntervals(final String dataSource, final List<Interval> intervals)
|
List<DataSegment> getUsedSegmentsForIntervals(final String dataSource, final List<Interval> intervals)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -120,6 +120,14 @@ public interface IndexerMetadataStorageCoordinator
|
|||||||
|
|
||||||
DataSourceMetadata getDataSourceMetadata(String dataSource);
|
DataSourceMetadata getDataSourceMetadata(String dataSource);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Removes entry for 'dataSource' from the dataSource metadata table.
|
||||||
|
*
|
||||||
|
* @param dataSource identifier
|
||||||
|
* @return true if the entry was deleted, false otherwise
|
||||||
|
*/
|
||||||
|
boolean deleteDataSourceMetadata(String dataSource);
|
||||||
|
|
||||||
void updateSegmentMetadata(Set<DataSegment> segments) throws IOException;
|
void updateSegmentMetadata(Set<DataSegment> segments) throws IOException;
|
||||||
|
|
||||||
void deleteSegments(Set<DataSegment> segments) throws IOException;
|
void deleteSegments(Set<DataSegment> segments) throws IOException;
|
||||||
|
@ -46,6 +46,9 @@ public class NoopSupervisorSpec implements SupervisorSpec
|
|||||||
{
|
{
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void reset() {}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -32,4 +32,6 @@ public interface Supervisor
|
|||||||
void stop(boolean stopGracefully);
|
void stop(boolean stopGracefully);
|
||||||
|
|
||||||
SupervisorReport getStatus();
|
SupervisorReport getStatus();
|
||||||
|
|
||||||
|
void reset();
|
||||||
}
|
}
|
||||||
|
@ -772,6 +772,26 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
|||||||
return retVal;
|
return retVal;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public boolean deleteDataSourceMetadata(final String dataSource)
|
||||||
|
{
|
||||||
|
return connector.retryWithHandle(
|
||||||
|
new HandleCallback<Boolean>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Boolean withHandle(Handle handle) throws Exception
|
||||||
|
{
|
||||||
|
int rows = handle.createStatement(
|
||||||
|
String.format("DELETE from %s WHERE dataSource = :dataSource", dbTables.getDataSourceTable())
|
||||||
|
)
|
||||||
|
.bind("dataSource", dataSource)
|
||||||
|
.execute();
|
||||||
|
|
||||||
|
return rows > 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
public void updateSegmentMetadata(final Set<DataSegment> segments) throws IOException
|
public void updateSegmentMetadata(final Set<DataSegment> segments) throws IOException
|
||||||
{
|
{
|
||||||
connector.getDBI().inTransaction(
|
connector.getDBI().inTransaction(
|
||||||
|
@ -523,4 +523,24 @@ public class IndexerSQLMetadataStorageCoordinatorTest
|
|||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testDeleteDataSourceMetadata() throws IOException
|
||||||
|
{
|
||||||
|
coordinator.announceHistoricalSegments(
|
||||||
|
ImmutableSet.of(defaultSegment),
|
||||||
|
new ObjectMetadata(null),
|
||||||
|
new ObjectMetadata(ImmutableMap.of("foo", "bar"))
|
||||||
|
);
|
||||||
|
|
||||||
|
Assert.assertEquals(
|
||||||
|
new ObjectMetadata(ImmutableMap.of("foo", "bar")),
|
||||||
|
coordinator.getDataSourceMetadata("fooDataSource")
|
||||||
|
);
|
||||||
|
|
||||||
|
Assert.assertFalse("deleteInvalidDataSourceMetadata", coordinator.deleteDataSourceMetadata("nonExistentDS"));
|
||||||
|
Assert.assertTrue("deleteValidDataSourceMetadata", coordinator.deleteDataSourceMetadata("fooDataSource"));
|
||||||
|
|
||||||
|
Assert.assertNull("getDataSourceMetadataNullAfterDelete", coordinator.getDataSourceMetadata("fooDataSource"));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
Loading…
x
Reference in New Issue
Block a user