Fix Cannot mark an unqueryable datasource's segments used / unused (#16127)

* * fix

* * address review comments

* * all remove the short-circuit for markUnused api

* * add test
This commit is contained in:
zachjsh 2024-03-15 17:25:02 -04:00 committed by GitHub
parent 3eefc47722
commit f3d77fe684
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
2 changed files with 102 additions and 112 deletions

View File

@ -166,12 +166,12 @@ public class DataSourcesResource
@Path("/{dataSourceName}") @Path("/{dataSourceName}")
@Produces(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON)
@ResourceFilters(DatasourceResourceFilter.class) @ResourceFilters(DatasourceResourceFilter.class)
public Response getDataSource( public Response getQueryableDataSource(
@PathParam("dataSourceName") final String dataSourceName, @PathParam("dataSourceName") final String dataSourceName,
@QueryParam("full") final String full @QueryParam("full") final String full
) )
{ {
final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); final ImmutableDruidDataSource dataSource = getQueryableDataSource(dataSourceName);
if (dataSource == null) { if (dataSource == null) {
return logAndCreateDataSourceNotFoundResponse(dataSourceName); return logAndCreateDataSourceNotFoundResponse(dataSourceName);
@ -209,7 +209,15 @@ public class DataSourcesResource
MarkDataSourceSegmentsPayload payload MarkDataSourceSegmentsPayload payload
) )
{ {
if (payload == null || !payload.isValid()) {
log.warn("Invalid request payload: [%s]", payload);
return Response
.status(Response.Status.BAD_REQUEST)
.entity("Invalid request payload, either interval or segmentIds array must be specified")
.build();
} else {
SegmentUpdateOperation operation = () -> { SegmentUpdateOperation operation = () -> {
final Interval interval = payload.getInterval(); final Interval interval = payload.getInterval();
if (interval != null) { if (interval != null) {
return segmentsMetadataManager.markAsUsedNonOvershadowedSegmentsInInterval(dataSourceName, interval); return segmentsMetadataManager.markAsUsedNonOvershadowedSegmentsInInterval(dataSourceName, interval);
@ -233,7 +241,9 @@ public class DataSourcesResource
return segmentsMetadataManager.markAsUsedNonOvershadowedSegments(dataSourceName, segmentIds); return segmentsMetadataManager.markAsUsedNonOvershadowedSegments(dataSourceName, segmentIds);
} }
}; };
return performSegmentUpdate(dataSourceName, payload, operation);
return performSegmentUpdate(dataSourceName, operation);
}
} }
@POST @POST
@ -247,6 +257,13 @@ public class DataSourcesResource
@Context final HttpServletRequest req @Context final HttpServletRequest req
) )
{ {
if (payload == null || !payload.isValid()) {
log.warn("Invalid request payload: [%s]", payload);
return Response
.status(Response.Status.BAD_REQUEST)
.entity("Invalid request payload, either interval or segmentIds array must be specified")
.build();
} else {
SegmentUpdateOperation operation = () -> { SegmentUpdateOperation operation = () -> {
final Interval interval = payload.getInterval(); final Interval interval = payload.getInterval();
final int numUpdatedSegments; final int numUpdatedSegments;
@ -278,30 +295,9 @@ public class DataSourcesResource
); );
return numUpdatedSegments; return numUpdatedSegments;
}; };
return performSegmentUpdate(dataSourceName, payload, operation);
}
private Response performSegmentUpdate(
String dataSourceName,
MarkDataSourceSegmentsPayload payload,
SegmentUpdateOperation operation
)
{
if (payload == null || !payload.isValid()) {
log.warn("Invalid request payload: [%s]", payload);
return Response
.status(Response.Status.BAD_REQUEST)
.entity("Invalid request payload, either interval or segmentIds array must be specified")
.build();
}
final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName);
if (dataSource == null) {
return logAndCreateDataSourceNotFoundResponse(dataSourceName);
}
return performSegmentUpdate(dataSourceName, operation); return performSegmentUpdate(dataSourceName, operation);
} }
}
private static Response logAndCreateDataSourceNotFoundResponse(String dataSourceName) private static Response logAndCreateDataSourceNotFoundResponse(String dataSourceName)
{ {
@ -434,7 +430,7 @@ public class DataSourcesResource
) )
{ {
if (simple == null && full == null) { if (simple == null && full == null) {
final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); final ImmutableDruidDataSource dataSource = getQueryableDataSource(dataSourceName);
if (dataSource == null) { if (dataSource == null) {
return logAndCreateDataSourceNotFoundResponse(dataSourceName); return logAndCreateDataSourceNotFoundResponse(dataSourceName);
} }
@ -460,7 +456,7 @@ public class DataSourcesResource
{ {
final Interval theInterval = Intervals.of(interval.replace('_', '/')); final Interval theInterval = Intervals.of(interval.replace('_', '/'));
if (simple == null && full == null) { if (simple == null && full == null) {
final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); final ImmutableDruidDataSource dataSource = getQueryableDataSource(dataSourceName);
if (dataSource == null) { if (dataSource == null) {
return logAndCreateDataSourceNotFoundResponse(dataSourceName); return logAndCreateDataSourceNotFoundResponse(dataSourceName);
} }
@ -617,7 +613,7 @@ public class DataSourcesResource
Predicate<Interval> intervalFilter Predicate<Interval> intervalFilter
) )
{ {
final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); final ImmutableDruidDataSource dataSource = getQueryableDataSource(dataSourceName);
if (dataSource == null) { if (dataSource == null) {
return logAndCreateDataSourceNotFoundResponse(dataSourceName); return logAndCreateDataSourceNotFoundResponse(dataSourceName);
@ -667,7 +663,7 @@ public class DataSourcesResource
@QueryParam("full") String full @QueryParam("full") String full
) )
{ {
ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); ImmutableDruidDataSource dataSource = getQueryableDataSource(dataSourceName);
if (dataSource == null) { if (dataSource == null) {
return logAndCreateDataSourceNotFoundResponse(dataSourceName); return logAndCreateDataSourceNotFoundResponse(dataSourceName);
} }
@ -689,7 +685,7 @@ public class DataSourcesResource
@PathParam("segmentId") String segmentId @PathParam("segmentId") String segmentId
) )
{ {
ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); ImmutableDruidDataSource dataSource = getQueryableDataSource(dataSourceName);
if (dataSource == null) { if (dataSource == null) {
return logAndCreateDataSourceNotFoundResponse(dataSourceName); return logAndCreateDataSourceNotFoundResponse(dataSourceName);
} }
@ -747,7 +743,7 @@ public class DataSourcesResource
} }
@Nullable @Nullable
private ImmutableDruidDataSource getDataSource(final String dataSourceName) private ImmutableDruidDataSource getQueryableDataSource(final String dataSourceName)
{ {
List<DruidDataSource> dataSources = serverInventoryView List<DruidDataSource> dataSources = serverInventoryView
.getInventory() .getInventory()

View File

@ -320,7 +320,7 @@ public class DataSourcesResourceTest
EasyMock.replay(inventoryView, server); EasyMock.replay(inventoryView, server);
DataSourcesResource dataSourcesResource = createResource(); DataSourcesResource dataSourcesResource = createResource();
Response response = dataSourcesResource.getDataSource("datasource1", "full"); Response response = dataSourcesResource.getQueryableDataSource("datasource1", "full");
ImmutableDruidDataSource result = (ImmutableDruidDataSource) response.getEntity(); ImmutableDruidDataSource result = (ImmutableDruidDataSource) response.getEntity();
Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(200, response.getStatus());
ImmutableDruidDataSourceTestUtils.assertEquals(dataSource1.toImmutableDruidDataSource(), result); ImmutableDruidDataSourceTestUtils.assertEquals(dataSource1.toImmutableDruidDataSource(), result);
@ -335,7 +335,7 @@ public class DataSourcesResourceTest
EasyMock.replay(inventoryView, server); EasyMock.replay(inventoryView, server);
DataSourcesResource dataSourcesResource = createResource(); DataSourcesResource dataSourcesResource = createResource();
Assert.assertEquals(204, dataSourcesResource.getDataSource("none", null).getStatus()); Assert.assertEquals(204, dataSourcesResource.getQueryableDataSource("none", null).getStatus());
EasyMock.verify(inventoryView, server); EasyMock.verify(inventoryView, server);
} }
@ -352,7 +352,7 @@ public class DataSourcesResourceTest
EasyMock.replay(inventoryView, server); EasyMock.replay(inventoryView, server);
DataSourcesResource dataSourcesResource = createResource(); DataSourcesResource dataSourcesResource = createResource();
Response response = dataSourcesResource.getDataSource("datasource1", null); Response response = dataSourcesResource.getQueryableDataSource("datasource1", null);
Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(200, response.getStatus());
Map<String, Map<String, Object>> result = (Map<String, Map<String, Object>>) response.getEntity(); Map<String, Map<String, Object>> result = (Map<String, Map<String, Object>>) response.getEntity();
Assert.assertEquals(1, ((Map) (result.get("tiers").get(null))).get("segmentCount")); Assert.assertEquals(1, ((Map) (result.get("tiers").get(null))).get("segmentCount"));
@ -385,7 +385,7 @@ public class DataSourcesResourceTest
EasyMock.replay(inventoryView, server, server2, server3); EasyMock.replay(inventoryView, server, server2, server3);
DataSourcesResource dataSourcesResource = createResource(); DataSourcesResource dataSourcesResource = createResource();
Response response = dataSourcesResource.getDataSource("datasource1", null); Response response = dataSourcesResource.getQueryableDataSource("datasource1", null);
Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(200, response.getStatus());
Map<String, Map<String, Object>> result = (Map<String, Map<String, Object>>) response.getEntity(); Map<String, Map<String, Object>> result = (Map<String, Map<String, Object>>) response.getEntity();
Assert.assertEquals(2, ((Map) (result.get("tiers").get("cold"))).get("segmentCount")); Assert.assertEquals(2, ((Map) (result.get("tiers").get("cold"))).get("segmentCount"));
@ -423,7 +423,7 @@ public class DataSourcesResourceTest
EasyMock.replay(inventoryView); EasyMock.replay(inventoryView);
DataSourcesResource dataSourcesResource = createResource(); DataSourcesResource dataSourcesResource = createResource();
Response response = dataSourcesResource.getDataSource("datasource1", null); Response response = dataSourcesResource.getQueryableDataSource("datasource1", null);
Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(200, response.getStatus());
Map<String, Map<String, Object>> result1 = (Map<String, Map<String, Object>>) response.getEntity(); Map<String, Map<String, Object>> result1 = (Map<String, Map<String, Object>>) response.getEntity();
Assert.assertEquals(2, ((Map) (result1.get("tiers").get("tier1"))).get("segmentCount")); Assert.assertEquals(2, ((Map) (result1.get("tiers").get("tier1"))).get("segmentCount"));
@ -438,7 +438,7 @@ public class DataSourcesResourceTest
Assert.assertEquals(30L, result1.get("segments").get("size")); Assert.assertEquals(30L, result1.get("segments").get("size"));
Assert.assertEquals(60L, result1.get("segments").get("replicatedSize")); Assert.assertEquals(60L, result1.get("segments").get("replicatedSize"));
response = dataSourcesResource.getDataSource("datasource2", null); response = dataSourcesResource.getQueryableDataSource("datasource2", null);
Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(200, response.getStatus());
Map<String, Map<String, Object>> result2 = (Map<String, Map<String, Object>>) response.getEntity(); Map<String, Map<String, Object>> result2 = (Map<String, Map<String, Object>>) response.getEntity();
Assert.assertEquals(1, ((Map) (result2.get("tiers").get("tier1"))).get("segmentCount")); Assert.assertEquals(1, ((Map) (result2.get("tiers").get("tier1"))).get("segmentCount"));
@ -733,13 +733,10 @@ public class DataSourcesResourceTest
@Test @Test
public void testMarkAsUsedNonOvershadowedSegmentsInterval() public void testMarkAsUsedNonOvershadowedSegmentsInterval()
{ {
DruidDataSource dataSource = new DruidDataSource("datasource1", new HashMap<>());
Interval interval = Intervals.of("2010-01-22/P1D"); Interval interval = Intervals.of("2010-01-22/P1D");
int numUpdatedSegments = int numUpdatedSegments =
segmentsMetadataManager.markAsUsedNonOvershadowedSegmentsInInterval(EasyMock.eq("datasource1"), EasyMock.eq(interval)); segmentsMetadataManager.markAsUsedNonOvershadowedSegmentsInInterval(EasyMock.eq("datasource1"), EasyMock.eq(interval));
EasyMock.expect(numUpdatedSegments).andReturn(3).once(); EasyMock.expect(numUpdatedSegments).andReturn(3).once();
EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once();
EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource).once();
EasyMock.replay(segmentsMetadataManager, inventoryView, server); EasyMock.replay(segmentsMetadataManager, inventoryView, server);
DataSourcesResource dataSourcesResource = createResource(); DataSourcesResource dataSourcesResource = createResource();
@ -754,13 +751,10 @@ public class DataSourcesResourceTest
@Test @Test
public void testMarkAsUsedNonOvershadowedSegmentsIntervalNoneUpdated() public void testMarkAsUsedNonOvershadowedSegmentsIntervalNoneUpdated()
{ {
DruidDataSource dataSource = new DruidDataSource("datasource1", new HashMap<>());
Interval interval = Intervals.of("2010-01-22/P1D"); Interval interval = Intervals.of("2010-01-22/P1D");
int numUpdatedSegments = int numUpdatedSegments =
segmentsMetadataManager.markAsUsedNonOvershadowedSegmentsInInterval(EasyMock.eq("datasource1"), EasyMock.eq(interval)); segmentsMetadataManager.markAsUsedNonOvershadowedSegmentsInInterval(EasyMock.eq("datasource1"), EasyMock.eq(interval));
EasyMock.expect(numUpdatedSegments).andReturn(0).once(); EasyMock.expect(numUpdatedSegments).andReturn(0).once();
EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once();
EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource).once();
EasyMock.replay(segmentsMetadataManager, inventoryView, server); EasyMock.replay(segmentsMetadataManager, inventoryView, server);
DataSourcesResource dataSourcesResource = createResource(); DataSourcesResource dataSourcesResource = createResource();
@ -776,13 +770,10 @@ public class DataSourcesResourceTest
@Test @Test
public void testMarkAsUsedNonOvershadowedSegmentsSet() public void testMarkAsUsedNonOvershadowedSegmentsSet()
{ {
DruidDataSource dataSource = new DruidDataSource("datasource1", new HashMap<>());
Set<String> segmentIds = ImmutableSet.of(dataSegmentList.get(1).getId().toString()); Set<String> segmentIds = ImmutableSet.of(dataSegmentList.get(1).getId().toString());
int numUpdatedSegments = int numUpdatedSegments =
segmentsMetadataManager.markAsUsedNonOvershadowedSegments(EasyMock.eq("datasource1"), EasyMock.eq(segmentIds)); segmentsMetadataManager.markAsUsedNonOvershadowedSegments(EasyMock.eq("datasource1"), EasyMock.eq(segmentIds));
EasyMock.expect(numUpdatedSegments).andReturn(3).once(); EasyMock.expect(numUpdatedSegments).andReturn(3).once();
EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once();
EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource).once();
EasyMock.replay(segmentsMetadataManager, inventoryView, server); EasyMock.replay(segmentsMetadataManager, inventoryView, server);
DataSourcesResource dataSourcesResource = createResource(); DataSourcesResource dataSourcesResource = createResource();
@ -798,13 +789,10 @@ public class DataSourcesResourceTest
@Test @Test
public void testMarkAsUsedNonOvershadowedSegmentsIntervalException() public void testMarkAsUsedNonOvershadowedSegmentsIntervalException()
{ {
DruidDataSource dataSource = new DruidDataSource("datasource1", new HashMap<>());
Interval interval = Intervals.of("2010-01-22/P1D"); Interval interval = Intervals.of("2010-01-22/P1D");
int numUpdatedSegments = int numUpdatedSegments =
segmentsMetadataManager.markAsUsedNonOvershadowedSegmentsInInterval(EasyMock.eq("datasource1"), EasyMock.eq(interval)); segmentsMetadataManager.markAsUsedNonOvershadowedSegmentsInInterval(EasyMock.eq("datasource1"), EasyMock.eq(interval));
EasyMock.expect(numUpdatedSegments).andThrow(new RuntimeException("Error!")).once(); EasyMock.expect(numUpdatedSegments).andThrow(new RuntimeException("Error!")).once();
EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once();
EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource).once();
EasyMock.replay(segmentsMetadataManager, inventoryView, server); EasyMock.replay(segmentsMetadataManager, inventoryView, server);
DataSourcesResource dataSourcesResource = createResource(); DataSourcesResource dataSourcesResource = createResource();
@ -820,8 +808,10 @@ public class DataSourcesResourceTest
@Test @Test
public void testMarkAsUsedNonOvershadowedSegmentsNoDataSource() public void testMarkAsUsedNonOvershadowedSegmentsNoDataSource()
{ {
EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); Interval interval = Intervals.of("2010-01-22/P1D");
EasyMock.expect(server.getDataSource("datasource1")).andReturn(null).once(); int numUpdatedSegments =
segmentsMetadataManager.markAsUsedNonOvershadowedSegmentsInInterval(EasyMock.eq("datasource1"), EasyMock.eq(interval));
EasyMock.expect(numUpdatedSegments).andReturn(0).once();
EasyMock.replay(segmentsMetadataManager, inventoryView, server); EasyMock.replay(segmentsMetadataManager, inventoryView, server);
DataSourcesResource dataSourcesResource = createResource(); DataSourcesResource dataSourcesResource = createResource();
@ -830,7 +820,8 @@ public class DataSourcesResourceTest
"datasource1", "datasource1",
new DataSourcesResource.MarkDataSourceSegmentsPayload(Intervals.of("2010-01-22/P1D"), null) new DataSourcesResource.MarkDataSourceSegmentsPayload(Intervals.of("2010-01-22/P1D"), null)
); );
Assert.assertEquals(204, response.getStatus()); Assert.assertEquals(200, response.getStatus());
Assert.assertEquals(ImmutableMap.of("numChangedSegments", 0), response.getEntity());
EasyMock.verify(segmentsMetadataManager); EasyMock.verify(segmentsMetadataManager);
} }
@ -1019,8 +1010,6 @@ public class DataSourcesResourceTest
.map(DataSegment::getId) .map(DataSegment::getId)
.collect(Collectors.toSet()); .collect(Collectors.toSet());
EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once();
EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource1).once();
EasyMock.expect(segmentsMetadataManager.markSegmentsAsUnused(segmentIds)).andReturn(1).once(); EasyMock.expect(segmentsMetadataManager.markSegmentsAsUnused(segmentIds)).andReturn(1).once();
EasyMock.replay(segmentsMetadataManager, inventoryView, server); EasyMock.replay(segmentsMetadataManager, inventoryView, server);
@ -1050,8 +1039,6 @@ public class DataSourcesResourceTest
.map(DataSegment::getId) .map(DataSegment::getId)
.collect(Collectors.toSet()); .collect(Collectors.toSet());
EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once();
EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource1).once();
EasyMock.expect(segmentsMetadataManager.markSegmentsAsUnused(segmentIds)).andReturn(0).once(); EasyMock.expect(segmentsMetadataManager.markSegmentsAsUnused(segmentIds)).andReturn(0).once();
EasyMock.replay(segmentsMetadataManager, inventoryView, server); EasyMock.replay(segmentsMetadataManager, inventoryView, server);
@ -1081,8 +1068,6 @@ public class DataSourcesResourceTest
.map(DataSegment::getId) .map(DataSegment::getId)
.collect(Collectors.toSet()); .collect(Collectors.toSet());
EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once();
EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource1).once();
EasyMock.expect(segmentsMetadataManager.markSegmentsAsUnused(segmentIds)) EasyMock.expect(segmentsMetadataManager.markSegmentsAsUnused(segmentIds))
.andThrow(new RuntimeException("Exception occurred")) .andThrow(new RuntimeException("Exception occurred"))
.once(); .once();
@ -1108,10 +1093,7 @@ public class DataSourcesResourceTest
public void testMarkAsUnusedSegmentsInInterval() public void testMarkAsUnusedSegmentsInInterval()
{ {
final Interval theInterval = Intervals.of("2010-01-01/P1D"); final Interval theInterval = Intervals.of("2010-01-01/P1D");
final DruidDataSource dataSource1 = new DruidDataSource("datasource1", new HashMap<>());
EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once();
EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource1).once();
EasyMock.expect(segmentsMetadataManager.markAsUnusedSegmentsInInterval("datasource1", theInterval)).andReturn(1).once(); EasyMock.expect(segmentsMetadataManager.markAsUnusedSegmentsInInterval("datasource1", theInterval)).andReturn(1).once();
EasyMock.replay(segmentsMetadataManager, inventoryView, server); EasyMock.replay(segmentsMetadataManager, inventoryView, server);
@ -1131,10 +1113,7 @@ public class DataSourcesResourceTest
public void testMarkAsUnusedSegmentsInIntervalNoChanges() public void testMarkAsUnusedSegmentsInIntervalNoChanges()
{ {
final Interval theInterval = Intervals.of("2010-01-01/P1D"); final Interval theInterval = Intervals.of("2010-01-01/P1D");
final DruidDataSource dataSource1 = new DruidDataSource("datasource1", new HashMap<>());
EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once();
EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource1).once();
EasyMock.expect(segmentsMetadataManager.markAsUnusedSegmentsInInterval("datasource1", theInterval)).andReturn(0).once(); EasyMock.expect(segmentsMetadataManager.markAsUnusedSegmentsInInterval("datasource1", theInterval)).andReturn(0).once();
EasyMock.replay(segmentsMetadataManager, inventoryView, server); EasyMock.replay(segmentsMetadataManager, inventoryView, server);
@ -1153,10 +1132,7 @@ public class DataSourcesResourceTest
public void testMarkAsUnusedSegmentsInIntervalException() public void testMarkAsUnusedSegmentsInIntervalException()
{ {
final Interval theInterval = Intervals.of("2010-01-01/P1D"); final Interval theInterval = Intervals.of("2010-01-01/P1D");
final DruidDataSource dataSource1 = new DruidDataSource("datasource1", new HashMap<>());
EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once();
EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource1).once();
EasyMock.expect(segmentsMetadataManager.markAsUnusedSegmentsInInterval("datasource1", theInterval)) EasyMock.expect(segmentsMetadataManager.markAsUnusedSegmentsInInterval("datasource1", theInterval))
.andThrow(new RuntimeException("Exception occurred")) .andThrow(new RuntimeException("Exception occurred"))
.once(); .once();
@ -1173,6 +1149,24 @@ public class DataSourcesResourceTest
EasyMock.verify(segmentsMetadataManager, inventoryView, server); EasyMock.verify(segmentsMetadataManager, inventoryView, server);
} }
@Test
public void testMarkAsUnusedSegmentsInIntervalNoDataSource()
{
final Interval theInterval = Intervals.of("2010-01-01/P1D");
EasyMock.expect(segmentsMetadataManager.markAsUnusedSegmentsInInterval("datasource1", theInterval)).andReturn(0).once();
EasyMock.replay(segmentsMetadataManager, inventoryView, server);
final DataSourcesResource.MarkDataSourceSegmentsPayload payload =
new DataSourcesResource.MarkDataSourceSegmentsPayload(theInterval, null);
DataSourcesResource dataSourcesResource = createResource();
prepareRequestForAudit();
Response response = dataSourcesResource.markSegmentsAsUnused("datasource1", payload, request);
Assert.assertEquals(200, response.getStatus());
Assert.assertEquals(ImmutableMap.of("numChangedSegments", 0), response.getEntity());
EasyMock.verify(segmentsMetadataManager);
}
@Test @Test
public void testMarkSegmentsAsUnusedNullPayload() public void testMarkSegmentsAsUnusedNullPayload()
{ {