Skip to content

Commit

Permalink
Return Total Result Count and Remaining Count in Get Snapshots Respon…
Browse files Browse the repository at this point in the history
…se (#76150)

Add total result count and remaining count to get snapshots response.
  • Loading branch information
original-brownbear committed Aug 9, 2021
1 parent 98d2989 commit a4983f5
Show file tree
Hide file tree
Showing 8 changed files with 170 additions and 86 deletions.
23 changes: 19 additions & 4 deletions docs/reference/snapshot-restore/apis/get-snapshot-api.asciidoc
Original file line number Diff line number Diff line change
Expand Up @@ -283,6 +283,15 @@ The snapshot `state` can be one of the following values:
If the request contained a size limit and there might be more results, a `next` field will be added to the response and can be used as the
`after` query parameter to fetch additional results.

`total`::
(integer)
The total number of snapshots that match the request when ignoring size limit or `after` query parameter.

`remaining`::
(integer)
The number of remaining snapshots that were not returned due to size limits and that can be fetched by additional requests using the `next`
field value.

[[get-snapshot-api-example]]
==== {api-examples-title}

Expand Down Expand Up @@ -322,7 +331,9 @@ The API returns the following response:
"successful": 0
}
}
]
],
"total": 1,
"remaining": 0
}
----
// TESTRESPONSE[s/"uuid": "vdRctLCxSketdKb54xw67g"/"uuid": $body.snapshots.0.uuid/]
Expand Down Expand Up @@ -392,10 +403,12 @@ The API returns the following response:
"total": 0,
"failed": 0,
"successful": 0
}
},
}
],
"next": "c25hcHNob3RfMixteV9yZXBvc2l0b3J5LHNuYXBzaG90XzI="
"next": "c25hcHNob3RfMixteV9yZXBvc2l0b3J5LHNuYXBzaG90XzI=",
"total": 3,
"remaining": 1
}
----
// TESTRESPONSE[s/"uuid": "dKb54xw67gvdRctLCxSket"/"uuid": $body.snapshots.0.uuid/]
Expand Down Expand Up @@ -449,7 +462,9 @@ The API returns the following response:
"successful": 0
}
}
]
],
"total": 3,
"remaining": 0
}
----
// TESTRESPONSE[s/"uuid": "dRctdKb54xw67gvLCxSket"/"uuid": $body.snapshots.0.uuid/]
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.core.Tuple;
import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.snapshots.AbstractSnapshotIntegTestCase;
import org.elasticsearch.snapshots.SnapshotInfo;
Expand Down Expand Up @@ -106,31 +105,34 @@ private void doTestPagination(String repoName,
GetSnapshotsRequest.SortBy sort,
SortOrder order) throws IOException {
final List<SnapshotInfo> allSnapshotsSorted = allSnapshotsSorted(names, repoName, sort, order);
final Tuple<String, List<SnapshotInfo>> batch1 = sortedWithLimit(repoName, sort, null, 2, order);
assertEquals(allSnapshotsSorted.subList(0, 2), batch1.v2());
final Tuple<String, List<SnapshotInfo>> batch2 = sortedWithLimit(repoName, sort, batch1.v1(), 2, order);
assertEquals(allSnapshotsSorted.subList(2, 4), batch2.v2());
final int lastBatch = names.size() - batch1.v2().size() - batch2.v2().size();
final Tuple<String, List<SnapshotInfo>> batch3 = sortedWithLimit(repoName, sort, batch2.v1(), lastBatch, order);
assertEquals(batch3.v2(), allSnapshotsSorted.subList(batch1.v2().size() + batch2.v2().size(), names.size()));
final Tuple<String, List<SnapshotInfo>> batch3NoLimit = sortedWithLimit(
final GetSnapshotsResponse batch1 = sortedWithLimit(repoName, sort, null, 2, order);
assertEquals(allSnapshotsSorted.subList(0, 2), batch1.getSnapshots());
final GetSnapshotsResponse batch2 = sortedWithLimit(repoName, sort, batch1.next(), 2, order);
assertEquals(allSnapshotsSorted.subList(2, 4), batch2.getSnapshots());
final int lastBatch = names.size() - batch1.getSnapshots().size() - batch2.getSnapshots().size();
final GetSnapshotsResponse batch3 = sortedWithLimit(repoName, sort, batch2.next(), lastBatch, order);
assertEquals(
batch3.getSnapshots(),
allSnapshotsSorted.subList(batch1.getSnapshots().size() + batch2.getSnapshots().size(), names.size())
);
final GetSnapshotsResponse batch3NoLimit = sortedWithLimit(
repoName,
sort,
batch2.v1(),
batch2.next(),
GetSnapshotsRequest.NO_LIMIT,
order
);
assertNull(batch3NoLimit.v1());
assertEquals(batch3.v2(), batch3NoLimit.v2());
final Tuple<String, List<SnapshotInfo>> batch3LargeLimit = sortedWithLimit(
assertNull(batch3NoLimit.next());
assertEquals(batch3.getSnapshots(), batch3NoLimit.getSnapshots());
final GetSnapshotsResponse batch3LargeLimit = sortedWithLimit(
repoName,
sort,
batch2.v1(),
batch2.next(),
lastBatch + randomIntBetween(1, 100),
order
);
assertEquals(batch3.v2(), batch3LargeLimit.v2());
assertNull(batch3LargeLimit.v1());
assertEquals(batch3.getSnapshots(), batch3LargeLimit.getSnapshots());
assertNull(batch3LargeLimit.next());
}

public void testSortAndPaginateWithInProgress() throws Exception {
Expand Down Expand Up @@ -180,16 +182,19 @@ private static void assertStablePagination(String repoName,
final List<SnapshotInfo> allSorted = allSnapshotsSorted(allSnapshotNames, repoName, sort, order);

for (int i = 1; i <= allSnapshotNames.size(); i++) {
final List<SnapshotInfo> subsetSorted = sortedWithLimit(repoName, sort, null, i, order).v2();
final List<SnapshotInfo> subsetSorted = sortedWithLimit(repoName, sort, null, i, order).getSnapshots();
assertEquals(subsetSorted, allSorted.subList(0, i));
}

for (int j = 0; j < allSnapshotNames.size(); j++) {
final SnapshotInfo after = allSorted.get(j);
for (int i = 1; i < allSnapshotNames.size() - j; i++) {
final List<SnapshotInfo> subsetSorted = sortedWithLimit(
repoName, sort, GetSnapshotsRequest.After.from(after, sort).asQueryParam(), i, order).v2();
final GetSnapshotsResponse getSnapshotsResponse =
sortedWithLimit(repoName, sort, GetSnapshotsRequest.After.from(after, sort).asQueryParam(), i, order);
final List<SnapshotInfo> subsetSorted = getSnapshotsResponse.getSnapshots();
assertEquals(subsetSorted, allSorted.subList(j + 1, j + i + 1));
assertEquals(allSnapshotNames.size(), getSnapshotsResponse.totalCount());
assertEquals(allSnapshotNames.size() - (j + i + 1), getSnapshotsResponse.remaining());
}
}
}
Expand All @@ -203,9 +208,11 @@ private static List<SnapshotInfo> allSnapshotsSorted(Collection<String> allSnaps
if (order == SortOrder.DESC || randomBoolean()) {
request.addParameter("order", order.toString());
}
final Response response = getRestClient().performRequest(request);
final List<SnapshotInfo> snapshotInfos = readSnapshotInfos(response).v2();
final GetSnapshotsResponse getSnapshotsResponse = readSnapshotInfos(getRestClient().performRequest(request));
final List<SnapshotInfo> snapshotInfos = getSnapshotsResponse.getSnapshots();
assertEquals(snapshotInfos.size(), allSnapshotNames.size());
assertEquals(getSnapshotsResponse.totalCount(), allSnapshotNames.size());
assertEquals(0, getSnapshotsResponse.remaining());
for (SnapshotInfo snapshotInfo : snapshotInfos) {
assertThat(snapshotInfo.snapshotId().getName(), is(in(allSnapshotNames)));
}
Expand All @@ -216,16 +223,15 @@ private static Request baseGetSnapshotsRequest(String repoName) {
return new Request(HttpGet.METHOD_NAME, "/_snapshot/" + repoName + "/*");
}

private static Tuple<String, List<SnapshotInfo>> readSnapshotInfos(Response response) throws IOException {
private static GetSnapshotsResponse readSnapshotInfos(Response response) throws IOException {
try (InputStream input = response.getEntity().getContent();
XContentParser parser = JsonXContent.jsonXContent.createParser(
NamedXContentRegistry.EMPTY, DeprecationHandler.THROW_UNSUPPORTED_OPERATION, input)) {
final GetSnapshotsResponse getSnapshotsResponse = GetSnapshotsResponse.fromXContent(parser);
return Tuple.tuple(getSnapshotsResponse.next(), getSnapshotsResponse.getSnapshots());
return GetSnapshotsResponse.fromXContent(parser);
}
}

private static Tuple<String, List<SnapshotInfo>> sortedWithLimit(String repoName,
private static GetSnapshotsResponse sortedWithLimit(String repoName,
GetSnapshotsRequest.SortBy sortBy,
String after,
int size,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@
import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsRequestBuilder;
import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsResponse;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.core.Tuple;
import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.threadpool.ThreadPool;

Expand Down Expand Up @@ -97,31 +96,28 @@ public void testResponseSizeLimit() throws Exception {

private void doTestPagination(String repoName, List<String> names, GetSnapshotsRequest.SortBy sort, SortOrder order) {
final List<SnapshotInfo> allSnapshotsSorted = allSnapshotsSorted(names, repoName, sort, order);
final Tuple<String, List<SnapshotInfo>> batch1 = sortedWithLimit(repoName, sort, null, 2, order);
assertEquals(allSnapshotsSorted.subList(0, 2), batch1.v2());
final Tuple<String, List<SnapshotInfo>> batch2 = sortedWithLimit(repoName, sort, batch1.v1(), 2, order);
assertEquals(allSnapshotsSorted.subList(2, 4), batch2.v2());
final int lastBatch = names.size() - batch1.v2().size() - batch2.v2().size();
final Tuple<String, List<SnapshotInfo>> batch3 = sortedWithLimit(repoName, sort, batch2.v1(), lastBatch, order);
assertEquals(batch3.v2(), allSnapshotsSorted.subList(batch1.v2().size() + batch2.v2().size(), names.size()));
final Tuple<String, List<SnapshotInfo>> batch3NoLimit = sortedWithLimit(
repoName,
sort,
batch2.v1(),
GetSnapshotsRequest.NO_LIMIT,
order
final GetSnapshotsResponse batch1 = sortedWithLimit(repoName, sort, null, 2, order);
assertEquals(allSnapshotsSorted.subList(0, 2), batch1.getSnapshots());
final GetSnapshotsResponse batch2 = sortedWithLimit(repoName, sort, batch1.next(), 2, order);
assertEquals(allSnapshotsSorted.subList(2, 4), batch2.getSnapshots());
final int lastBatch = names.size() - batch1.getSnapshots().size() - batch2.getSnapshots().size();
final GetSnapshotsResponse batch3 = sortedWithLimit(repoName, sort, batch2.next(), lastBatch, order);
assertEquals(
batch3.getSnapshots(),
allSnapshotsSorted.subList(batch1.getSnapshots().size() + batch2.getSnapshots().size(), names.size())
);
assertNull(batch3NoLimit.v1());
assertEquals(batch3.v2(), batch3NoLimit.v2());
final Tuple<String, List<SnapshotInfo>> batch3LargeLimit = sortedWithLimit(
final GetSnapshotsResponse batch3NoLimit = sortedWithLimit(repoName, sort, batch2.next(), GetSnapshotsRequest.NO_LIMIT, order);
assertNull(batch3NoLimit.next());
assertEquals(batch3.getSnapshots(), batch3NoLimit.getSnapshots());
final GetSnapshotsResponse batch3LargeLimit = sortedWithLimit(
repoName,
sort,
batch2.v1(),
batch2.next(),
lastBatch + randomIntBetween(1, 100),
order
);
assertEquals(batch3.v2(), batch3LargeLimit.v2());
assertNull(batch3LargeLimit.v1());
assertEquals(batch3.getSnapshots(), batch3LargeLimit.getSnapshots());
assertNull(batch3LargeLimit.next());
}

public void testSortAndPaginateWithInProgress() throws Exception {
Expand Down Expand Up @@ -187,20 +183,24 @@ private static void assertStablePagination(String repoName, Collection<String> a
final List<SnapshotInfo> allSorted = allSnapshotsSorted(allSnapshotNames, repoName, sort, order);

for (int i = 1; i <= allSnapshotNames.size(); i++) {
final Tuple<String, List<SnapshotInfo>> subsetSorted = sortedWithLimit(repoName, sort, null, i, order);
assertEquals(allSorted.subList(0, i), subsetSorted.v2());
final GetSnapshotsResponse subsetSorted = sortedWithLimit(repoName, sort, null, i, order);
assertEquals(allSorted.subList(0, i), subsetSorted.getSnapshots());
}

for (int j = 0; j < allSnapshotNames.size(); j++) {
final SnapshotInfo after = allSorted.get(j);
for (int i = 1; i < allSnapshotNames.size() - j; i++) {
final List<SnapshotInfo> subsetSorted = sortedWithLimit(
final GetSnapshotsResponse getSnapshotsResponse = sortedWithLimit(
repoName,
sort,
GetSnapshotsRequest.After.from(after, sort).asQueryParam(),
i,
order
).v2();
);
final List<SnapshotInfo> subsetSorted = getSnapshotsResponse.getSnapshots();
assertEquals(subsetSorted, allSorted.subList(j + 1, j + i + 1));
assertEquals(allSnapshotNames.size(), getSnapshotsResponse.totalCount());
assertEquals(allSnapshotNames.size() - (j + i + 1), getSnapshotsResponse.remaining());
assertEquals(subsetSorted, allSorted.subList(j + 1, j + i + 1));
}
}
Expand All @@ -212,15 +212,18 @@ private static List<SnapshotInfo> allSnapshotsSorted(
GetSnapshotsRequest.SortBy sortBy,
SortOrder order
) {
final List<SnapshotInfo> snapshotInfos = sortedWithLimit(repoName, sortBy, null, GetSnapshotsRequest.NO_LIMIT, order).v2();
final GetSnapshotsResponse getSnapshotsResponse = sortedWithLimit(repoName, sortBy, null, GetSnapshotsRequest.NO_LIMIT, order);
final List<SnapshotInfo> snapshotInfos = getSnapshotsResponse.getSnapshots();
assertEquals(snapshotInfos.size(), allSnapshotNames.size());
assertEquals(getSnapshotsResponse.totalCount(), allSnapshotNames.size());
assertEquals(0, getSnapshotsResponse.remaining());
for (SnapshotInfo snapshotInfo : snapshotInfos) {
assertThat(snapshotInfo.snapshotId().getName(), is(in(allSnapshotNames)));
}
return snapshotInfos;
}

private static Tuple<String, List<SnapshotInfo>> sortedWithLimit(
private static GetSnapshotsResponse sortedWithLimit(
String repoName,
GetSnapshotsRequest.SortBy sortBy,
String after,
Expand All @@ -232,7 +235,7 @@ private static Tuple<String, List<SnapshotInfo>> sortedWithLimit(
.setSize(size)
.setOrder(order)
.get();
return Tuple.tuple(response.next(), response.getSnapshots());
return response;
}

private static GetSnapshotsRequestBuilder baseGetSnapshotsRequest(String repoName) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,8 @@ public class GetSnapshotsRequest extends MasterNodeRequest<GetSnapshotsRequest>

public static final Version PAGINATED_GET_SNAPSHOTS_VERSION = Version.V_7_14_0;

public static final Version NUMERIC_PAGINATION_VERSION = Version.V_8_0_0;

public static final int NO_LIMIT = -1;

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,11 +34,19 @@
*/
public class GetSnapshotsResponse extends ActionResponse implements ToXContentObject {

private static final int UNKNOWN_COUNT = -1;

@SuppressWarnings("unchecked")
private static final ConstructingObjectParser<GetSnapshotsResponse, Void> GET_SNAPSHOT_PARSER = new ConstructingObjectParser<>(
GetSnapshotsResponse.class.getName(),
true,
(args) -> new GetSnapshotsResponse((List<SnapshotInfo>) args[0], (Map<String, ElasticsearchException>) args[1], (String) args[2])
(args) -> new GetSnapshotsResponse(
(List<SnapshotInfo>) args[0],
(Map<String, ElasticsearchException>) args[1],
(String) args[2],
args[3] == null ? UNKNOWN_COUNT : (int) args[3],
args[4] == null ? UNKNOWN_COUNT : (int) args[4]
)
);

static {
Expand All @@ -53,6 +61,8 @@ public class GetSnapshotsResponse extends ActionResponse implements ToXContentOb
new ParseField("failures")
);
GET_SNAPSHOT_PARSER.declareStringOrNull(ConstructingObjectParser.optionalConstructorArg(), new ParseField("next"));
GET_SNAPSHOT_PARSER.declareIntOrNull(ConstructingObjectParser.optionalConstructorArg(), UNKNOWN_COUNT, new ParseField("total"));
GET_SNAPSHOT_PARSER.declareIntOrNull(ConstructingObjectParser.optionalConstructorArg(), UNKNOWN_COUNT, new ParseField("remaining"));
}

private final List<SnapshotInfo> snapshots;
Expand All @@ -62,10 +72,22 @@ public class GetSnapshotsResponse extends ActionResponse implements ToXContentOb
@Nullable
private final String next;

public GetSnapshotsResponse(List<SnapshotInfo> snapshots, Map<String, ElasticsearchException> failures, @Nullable String next) {
private final int total;

private final int remaining;

public GetSnapshotsResponse(
List<SnapshotInfo> snapshots,
Map<String, ElasticsearchException> failures,
@Nullable String next,
final int total,
final int remaining
) {
this.snapshots = List.copyOf(snapshots);
this.failures = failures == null ? Map.of() : Map.copyOf(failures);
this.next = next;
this.total = total;
this.remaining = remaining;
}

public GetSnapshotsResponse(StreamInput in) throws IOException {
Expand All @@ -78,6 +100,13 @@ public GetSnapshotsResponse(StreamInput in) throws IOException {
this.failures = Collections.emptyMap();
this.next = null;
}
if (in.getVersion().onOrAfter(GetSnapshotsRequest.NUMERIC_PAGINATION_VERSION)) {
this.total = in.readVInt();
this.remaining = in.readVInt();
} else {
this.total = UNKNOWN_COUNT;
this.remaining = UNKNOWN_COUNT;
}
}

/**
Expand Down Expand Up @@ -108,6 +137,14 @@ public boolean isFailed() {
return failures.isEmpty() == false;
}

public int totalCount() {
return total;
}

public int remaining() {
return remaining;
}

@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeList(snapshots);
Expand All @@ -120,6 +157,10 @@ public void writeTo(StreamOutput out) throws IOException {
throw failures.values().iterator().next();
}
}
if (out.getVersion().onOrAfter(GetSnapshotsRequest.NUMERIC_PAGINATION_VERSION)) {
out.writeVInt(total);
out.writeVInt(remaining);
}
}

@Override
Expand All @@ -145,6 +186,12 @@ public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params par
if (next != null) {
builder.field("next", next);
}
if (total >= 0) {
builder.field("total", total);
}
if (remaining >= 0) {
builder.field("remaining", remaining);
}
builder.endObject();
return builder;
}
Expand Down
Loading

0 comments on commit a4983f5

Please sign in to comment.