This is an automated email from the ASF dual-hosted git repository.

abhishekrb pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/druid.git


The following commit(s) were added to refs/heads/master by this push:
     new f3d77fe6843 Fix Cannot mark an unqueryable datasource's segments used 
/ unused (#16127)
f3d77fe6843 is described below

commit f3d77fe6843dbd753a5f5a362467d2c878734094
Author: zachjsh <[email protected]>
AuthorDate: Fri Mar 15 17:25:02 2024 -0400

    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
---
 .../druid/server/http/DataSourcesResource.java     | 148 ++++++++++-----------
 .../druid/server/http/DataSourcesResourceTest.java |  66 +++++----
 2 files changed, 102 insertions(+), 112 deletions(-)

diff --git 
a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java 
b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java
index b83bda21dcd..d0458243fcf 100644
--- a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java
+++ b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java
@@ -166,12 +166,12 @@ public class DataSourcesResource
   @Path("/{dataSourceName}")
   @Produces(MediaType.APPLICATION_JSON)
   @ResourceFilters(DatasourceResourceFilter.class)
-  public Response getDataSource(
+  public Response getQueryableDataSource(
       @PathParam("dataSourceName") final String dataSourceName,
       @QueryParam("full") final String full
   )
   {
-    final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName);
+    final ImmutableDruidDataSource dataSource = 
getQueryableDataSource(dataSourceName);
 
     if (dataSource == null) {
       return logAndCreateDataSourceNotFoundResponse(dataSourceName);
@@ -209,31 +209,41 @@ public class DataSourcesResource
       MarkDataSourceSegmentsPayload payload
   )
   {
-    SegmentUpdateOperation operation = () -> {
-      final Interval interval = payload.getInterval();
-      if (interval != null) {
-        return 
segmentsMetadataManager.markAsUsedNonOvershadowedSegmentsInInterval(dataSourceName,
 interval);
-      } else {
-        final Set<String> segmentIds = payload.getSegmentIds();
-        if (segmentIds == null || segmentIds.isEmpty()) {
-          return 0;
-        }
+    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 = () -> {
+
+        final Interval interval = payload.getInterval();
+        if (interval != null) {
+          return 
segmentsMetadataManager.markAsUsedNonOvershadowedSegmentsInInterval(dataSourceName,
 interval);
+        } else {
+          final Set<String> segmentIds = payload.getSegmentIds();
+          if (segmentIds == null || segmentIds.isEmpty()) {
+            return 0;
+          }
 
-        // Validate segmentIds
-        final List<String> invalidSegmentIds = new ArrayList<>();
-        for (String segmentId : segmentIds) {
-          if (SegmentId.iteratePossibleParsingsWithDataSource(dataSourceName, 
segmentId).isEmpty()) {
-            invalidSegmentIds.add(segmentId);
+          // Validate segmentIds
+          final List<String> invalidSegmentIds = new ArrayList<>();
+          for (String segmentId : segmentIds) {
+            if 
(SegmentId.iteratePossibleParsingsWithDataSource(dataSourceName, 
segmentId).isEmpty()) {
+              invalidSegmentIds.add(segmentId);
+            }
           }
+          if (!invalidSegmentIds.isEmpty()) {
+            throw InvalidInput.exception("Could not parse invalid segment 
IDs[%s]", invalidSegmentIds);
+          }
+
+          return 
segmentsMetadataManager.markAsUsedNonOvershadowedSegments(dataSourceName, 
segmentIds);
         }
-        if (!invalidSegmentIds.isEmpty()) {
-          throw InvalidInput.exception("Could not parse invalid segment 
IDs[%s]", invalidSegmentIds);
-        }
+      };
 
-        return 
segmentsMetadataManager.markAsUsedNonOvershadowedSegments(dataSourceName, 
segmentIds);
-      }
-    };
-    return performSegmentUpdate(dataSourceName, payload, operation);
+      return performSegmentUpdate(dataSourceName, operation);
+    }
   }
 
   @POST
@@ -246,46 +256,6 @@ public class DataSourcesResource
       final MarkDataSourceSegmentsPayload payload,
       @Context final HttpServletRequest req
   )
-  {
-    SegmentUpdateOperation operation = () -> {
-      final Interval interval = payload.getInterval();
-      final int numUpdatedSegments;
-      if (interval != null) {
-        numUpdatedSegments = 
segmentsMetadataManager.markAsUnusedSegmentsInInterval(dataSourceName, 
interval);
-      } else {
-        final Set<SegmentId> segmentIds =
-            payload.getSegmentIds()
-                   .stream()
-                   .map(idStr -> SegmentId.tryParse(dataSourceName, idStr))
-                   .filter(Objects::nonNull)
-                   .collect(Collectors.toSet());
-
-        // Filter out segmentIds that do not belong to this datasource
-        numUpdatedSegments = segmentsMetadataManager.markSegmentsAsUnused(
-            segmentIds.stream()
-                      .filter(segmentId -> 
segmentId.getDataSource().equals(dataSourceName))
-                      .collect(Collectors.toSet())
-        );
-      }
-      auditManager.doAudit(
-          AuditEntry.builder()
-                    .key(dataSourceName)
-                    .type("segment.markUnused")
-                    .payload(payload)
-                    .auditInfo(AuthorizationUtils.buildAuditInfo(req))
-                    
.request(AuthorizationUtils.buildRequestInfo("coordinator", req))
-                    .build()
-      );
-      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);
@@ -293,14 +263,40 @@ public class DataSourcesResource
           .status(Response.Status.BAD_REQUEST)
           .entity("Invalid request payload, either interval or segmentIds 
array must be specified")
           .build();
+    } else {
+      SegmentUpdateOperation operation = () -> {
+        final Interval interval = payload.getInterval();
+        final int numUpdatedSegments;
+        if (interval != null) {
+          numUpdatedSegments = 
segmentsMetadataManager.markAsUnusedSegmentsInInterval(dataSourceName, 
interval);
+        } else {
+          final Set<SegmentId> segmentIds =
+              payload.getSegmentIds()
+                  .stream()
+                  .map(idStr -> SegmentId.tryParse(dataSourceName, idStr))
+                  .filter(Objects::nonNull)
+                  .collect(Collectors.toSet());
+
+          // Filter out segmentIds that do not belong to this datasource
+          numUpdatedSegments = segmentsMetadataManager.markSegmentsAsUnused(
+              segmentIds.stream()
+                  .filter(segmentId -> 
segmentId.getDataSource().equals(dataSourceName))
+                  .collect(Collectors.toSet())
+          );
+        }
+        auditManager.doAudit(
+            AuditEntry.builder()
+                .key(dataSourceName)
+                .type("segment.markUnused")
+                .payload(payload)
+                .auditInfo(AuthorizationUtils.buildAuditInfo(req))
+                .request(AuthorizationUtils.buildRequestInfo("coordinator", 
req))
+                .build()
+        );
+        return numUpdatedSegments;
+      };
+      return performSegmentUpdate(dataSourceName, operation);
     }
-
-    final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName);
-    if (dataSource == null) {
-      return logAndCreateDataSourceNotFoundResponse(dataSourceName);
-    }
-
-    return performSegmentUpdate(dataSourceName, operation);
   }
 
   private static Response logAndCreateDataSourceNotFoundResponse(String 
dataSourceName)
@@ -434,7 +430,7 @@ public class DataSourcesResource
   )
   {
     if (simple == null && full == null) {
-      final ImmutableDruidDataSource dataSource = 
getDataSource(dataSourceName);
+      final ImmutableDruidDataSource dataSource = 
getQueryableDataSource(dataSourceName);
       if (dataSource == null) {
         return logAndCreateDataSourceNotFoundResponse(dataSourceName);
       }
@@ -460,7 +456,7 @@ public class DataSourcesResource
   {
     final Interval theInterval = Intervals.of(interval.replace('_', '/'));
     if (simple == null && full == null) {
-      final ImmutableDruidDataSource dataSource = 
getDataSource(dataSourceName);
+      final ImmutableDruidDataSource dataSource = 
getQueryableDataSource(dataSourceName);
       if (dataSource == null) {
         return logAndCreateDataSourceNotFoundResponse(dataSourceName);
       }
@@ -617,7 +613,7 @@ public class DataSourcesResource
       Predicate<Interval> intervalFilter
   )
   {
-    final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName);
+    final ImmutableDruidDataSource dataSource = 
getQueryableDataSource(dataSourceName);
 
     if (dataSource == null) {
       return logAndCreateDataSourceNotFoundResponse(dataSourceName);
@@ -667,7 +663,7 @@ public class DataSourcesResource
       @QueryParam("full") String full
   )
   {
-    ImmutableDruidDataSource dataSource = getDataSource(dataSourceName);
+    ImmutableDruidDataSource dataSource = 
getQueryableDataSource(dataSourceName);
     if (dataSource == null) {
       return logAndCreateDataSourceNotFoundResponse(dataSourceName);
     }
@@ -689,7 +685,7 @@ public class DataSourcesResource
       @PathParam("segmentId") String segmentId
   )
   {
-    ImmutableDruidDataSource dataSource = getDataSource(dataSourceName);
+    ImmutableDruidDataSource dataSource = 
getQueryableDataSource(dataSourceName);
     if (dataSource == null) {
       return logAndCreateDataSourceNotFoundResponse(dataSourceName);
     }
@@ -747,7 +743,7 @@ public class DataSourcesResource
   }
 
   @Nullable
-  private ImmutableDruidDataSource getDataSource(final String dataSourceName)
+  private ImmutableDruidDataSource getQueryableDataSource(final String 
dataSourceName)
   {
     List<DruidDataSource> dataSources = serverInventoryView
         .getInventory()
diff --git 
a/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java
 
b/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java
index 69fe23a75b5..ace68ad3c21 100644
--- 
a/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java
+++ 
b/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java
@@ -320,7 +320,7 @@ public class DataSourcesResourceTest
 
     EasyMock.replay(inventoryView, server);
     DataSourcesResource dataSourcesResource = createResource();
-    Response response = dataSourcesResource.getDataSource("datasource1", 
"full");
+    Response response = 
dataSourcesResource.getQueryableDataSource("datasource1", "full");
     ImmutableDruidDataSource result = (ImmutableDruidDataSource) 
response.getEntity();
     Assert.assertEquals(200, response.getStatus());
     
ImmutableDruidDataSourceTestUtils.assertEquals(dataSource1.toImmutableDruidDataSource(),
 result);
@@ -335,7 +335,7 @@ public class DataSourcesResourceTest
 
     EasyMock.replay(inventoryView, server);
     DataSourcesResource dataSourcesResource = createResource();
-    Assert.assertEquals(204, dataSourcesResource.getDataSource("none", 
null).getStatus());
+    Assert.assertEquals(204, 
dataSourcesResource.getQueryableDataSource("none", null).getStatus());
     EasyMock.verify(inventoryView, server);
   }
 
@@ -352,7 +352,7 @@ public class DataSourcesResourceTest
 
     EasyMock.replay(inventoryView, server);
     DataSourcesResource dataSourcesResource = createResource();
-    Response response = dataSourcesResource.getDataSource("datasource1", null);
+    Response response = 
dataSourcesResource.getQueryableDataSource("datasource1", null);
     Assert.assertEquals(200, response.getStatus());
     Map<String, Map<String, Object>> result = (Map<String, Map<String, 
Object>>) response.getEntity();
     Assert.assertEquals(1, ((Map) 
(result.get("tiers").get(null))).get("segmentCount"));
@@ -385,7 +385,7 @@ public class DataSourcesResourceTest
 
     EasyMock.replay(inventoryView, server, server2, server3);
     DataSourcesResource dataSourcesResource = createResource();
-    Response response = dataSourcesResource.getDataSource("datasource1", null);
+    Response response = 
dataSourcesResource.getQueryableDataSource("datasource1", null);
     Assert.assertEquals(200, response.getStatus());
     Map<String, Map<String, Object>> result = (Map<String, Map<String, 
Object>>) response.getEntity();
     Assert.assertEquals(2, ((Map) 
(result.get("tiers").get("cold"))).get("segmentCount"));
@@ -423,7 +423,7 @@ public class DataSourcesResourceTest
     EasyMock.replay(inventoryView);
 
     DataSourcesResource dataSourcesResource = createResource();
-    Response response = dataSourcesResource.getDataSource("datasource1", null);
+    Response response = 
dataSourcesResource.getQueryableDataSource("datasource1", null);
     Assert.assertEquals(200, response.getStatus());
     Map<String, Map<String, Object>> result1 = (Map<String, Map<String, 
Object>>) response.getEntity();
     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(60L, result1.get("segments").get("replicatedSize"));
 
-    response = dataSourcesResource.getDataSource("datasource2", null);
+    response = dataSourcesResource.getQueryableDataSource("datasource2", null);
     Assert.assertEquals(200, response.getStatus());
     Map<String, Map<String, Object>> result2 = (Map<String, Map<String, 
Object>>) response.getEntity();
     Assert.assertEquals(1, ((Map) 
(result2.get("tiers").get("tier1"))).get("segmentCount"));
@@ -733,13 +733,10 @@ public class DataSourcesResourceTest
   @Test
   public void testMarkAsUsedNonOvershadowedSegmentsInterval()
   {
-    DruidDataSource dataSource = new DruidDataSource("datasource1", new 
HashMap<>());
     Interval interval = Intervals.of("2010-01-22/P1D");
     int numUpdatedSegments =
         
segmentsMetadataManager.markAsUsedNonOvershadowedSegmentsInInterval(EasyMock.eq("datasource1"),
 EasyMock.eq(interval));
     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);
 
     DataSourcesResource dataSourcesResource = createResource();
@@ -754,13 +751,10 @@ public class DataSourcesResourceTest
   @Test
   public void testMarkAsUsedNonOvershadowedSegmentsIntervalNoneUpdated()
   {
-    DruidDataSource dataSource = new DruidDataSource("datasource1", new 
HashMap<>());
     Interval interval = Intervals.of("2010-01-22/P1D");
     int numUpdatedSegments =
         
segmentsMetadataManager.markAsUsedNonOvershadowedSegmentsInInterval(EasyMock.eq("datasource1"),
 EasyMock.eq(interval));
     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);
 
     DataSourcesResource dataSourcesResource = createResource();
@@ -776,13 +770,10 @@ public class DataSourcesResourceTest
   @Test
   public void testMarkAsUsedNonOvershadowedSegmentsSet()
   {
-    DruidDataSource dataSource = new DruidDataSource("datasource1", new 
HashMap<>());
     Set<String> segmentIds = 
ImmutableSet.of(dataSegmentList.get(1).getId().toString());
     int numUpdatedSegments =
         
segmentsMetadataManager.markAsUsedNonOvershadowedSegments(EasyMock.eq("datasource1"),
 EasyMock.eq(segmentIds));
     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);
 
     DataSourcesResource dataSourcesResource = createResource();
@@ -798,13 +789,10 @@ public class DataSourcesResourceTest
   @Test
   public void testMarkAsUsedNonOvershadowedSegmentsIntervalException()
   {
-    DruidDataSource dataSource = new DruidDataSource("datasource1", new 
HashMap<>());
     Interval interval = Intervals.of("2010-01-22/P1D");
     int numUpdatedSegments =
         
segmentsMetadataManager.markAsUsedNonOvershadowedSegmentsInInterval(EasyMock.eq("datasource1"),
 EasyMock.eq(interval));
     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);
 
     DataSourcesResource dataSourcesResource = createResource();
@@ -820,8 +808,10 @@ public class DataSourcesResourceTest
   @Test
   public void testMarkAsUsedNonOvershadowedSegmentsNoDataSource()
   {
-    
EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once();
-    
EasyMock.expect(server.getDataSource("datasource1")).andReturn(null).once();
+    Interval interval = Intervals.of("2010-01-22/P1D");
+    int numUpdatedSegments =
+        
segmentsMetadataManager.markAsUsedNonOvershadowedSegmentsInInterval(EasyMock.eq("datasource1"),
 EasyMock.eq(interval));
+    EasyMock.expect(numUpdatedSegments).andReturn(0).once();
     EasyMock.replay(segmentsMetadataManager, inventoryView, server);
 
     DataSourcesResource dataSourcesResource = createResource();
@@ -830,7 +820,8 @@ public class DataSourcesResourceTest
         "datasource1",
         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);
   }
 
@@ -1019,8 +1010,6 @@ public class DataSourcesResourceTest
                        .map(DataSegment::getId)
                        .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.replay(segmentsMetadataManager, inventoryView, server);
 
@@ -1050,8 +1039,6 @@ public class DataSourcesResourceTest
                        .map(DataSegment::getId)
                        .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.replay(segmentsMetadataManager, inventoryView, server);
 
@@ -1081,8 +1068,6 @@ public class DataSourcesResourceTest
                        .map(DataSegment::getId)
                        .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))
             .andThrow(new RuntimeException("Exception occurred"))
             .once();
@@ -1108,10 +1093,7 @@ public class DataSourcesResourceTest
   public void testMarkAsUnusedSegmentsInInterval()
   {
     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.replay(segmentsMetadataManager, inventoryView, server);
 
@@ -1131,10 +1113,7 @@ public class DataSourcesResourceTest
   public void testMarkAsUnusedSegmentsInIntervalNoChanges()
   {
     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.replay(segmentsMetadataManager, inventoryView, server);
 
@@ -1153,10 +1132,7 @@ public class DataSourcesResourceTest
   public void testMarkAsUnusedSegmentsInIntervalException()
   {
     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))
             .andThrow(new RuntimeException("Exception occurred"))
             .once();
@@ -1173,6 +1149,24 @@ public class DataSourcesResourceTest
     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
   public void testMarkSegmentsAsUnusedNullPayload()
   {


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to