Skip to content

Conversation

@quux00
Copy link
Contributor

@quux00 quux00 commented Jun 2, 2023

During a cross cluster search, the _cluster object in the SearchResponse will have a non-zero total count and thus is useful status information for the user. That information is included in the _async_search/<id> endpoint already because it is part of the underlying SearchResponse.

However, that information is not present in the _async_search/status/<id> endpoint, since that does not show the SearchResponse details. With a cross-cluster search this is useful information to understand (1) how many clusters are being searched; (2) whether they are successful or skipped, so it is being added in this commit.

When an async search is performed against the local cluster only , the _cluster object will not be present in the AsyncStatusResponse, since it is not informative (TransportSearchAction uses Clusters.EMPTY for executeLocalSearch).

@quux00 quux00 added >non-issue :Search/Search Search-related issues that do not fall into other categories Team:Search Meta label for search team v8.9.0 labels Jun 2, 2023
@quux00 quux00 requested a review from javanna June 2, 2023 19:31
@elasticsearchmachine
Copy link
Collaborator

Pinging @elastic/es-search (Team:Search)

@quux00
Copy link
Contributor Author

quux00 commented Jun 2, 2023

Manual testing examples:

Local cluster only searches do not show the _clusters object from either from the _async_search or the _async_search/status endpoints:

POST blogs/_async_search?ccs_minimize_roundtrips=true
{
  "id": "Fjl2N3plbHRMVDl5Q29KamF6N3NVZWcbTXBCOEliLUdSdzZSZmZkWW15ZmdNQToyMDAx",
  "is_partial": true,
  "is_running": true,
  "start_time_in_millis": 1685725885651,
  "expiration_time_in_millis": 1686157885651,
  "response": {
    "took": 1028,
    "timed_out": false,
    "terminated_early": false,
    "num_reduce_phases": 0,
    "_shards": {
      "total": 3,
      "successful": 0,
      "skipped": 0,
      "failed": 0
    },
    "hits": {
      "total": {
        "value": 0,
        "relation": "gte"
      },
      "max_score": null,
      "hits": []
    }
  }
}

// intermediate
GET /_async_search/status/Fjl2N3plbHRMVDl5Q29KamF6N3NVZWcbTXBCOEliLUdSdzZSZmZkWW15ZmdNQToyMDAx
{
  "id": "Fjl2N3plbHRMVDl5Q29KamF6N3NVZWcbTXBCOEliLUdSdzZSZmZkWW15ZmdNQToyMDAx",
  "is_running": true,
  "is_partial": true,
  "start_time_in_millis": 1685725966337,
  "expiration_time_in_millis": 1686157966337,
  "_shards": {
    "total": 3,
    "successful": 0,
    "skipped": 0,
    "failed": 0
  }
}


// final
GET /_async_search/status/Fjl2N3plbHRMVDl5Q29KamF6N3NVZWcbTXBCOEliLUdSdzZSZmZkWW15ZmdNQToyMDAx

{
  "id": "Fjl2N3plbHRMVDl5Q29KamF6N3NVZWcbTXBCOEliLUdSdzZSZmZkWW15ZmdNQToyMDAx",
  "is_running": false,
  "is_partial": false,
  "start_time_in_millis": 1685725885651 ,
  "expiration_time_in_millis": 1686157885651,
  "_shards": {
    "total": 3,
    "successful": 3,
    "skipped": 0,
    "failed": 0
  },
  "completion_status": 200
}

whereas searches with one or more remote clusters (CCS) shows the _cluster object from both endpoints:

POST remote1:*,remote2:*/_async_search?ccs_minimize_roundtrips=true

{
  "id": "FlRYVE81OWZDUm8tMmN3RjQtNEw3SEEbTXCOEliLUdSdzZSZmZkWW15ZmdNQToyMjMz",
  "is_partial": true,
  "is_running": true,
  "start_time_in_millis": 1685726121946,
  "expiration_time_in_millis": 168615812194,
  "response": {
    "took": 1005,
    "timed_out": false,
    "terminated_early": false,
    "num_reduce_phases": 0,
    "_shards": {
      "total": 0,
      "successful": 0,
      "skipped": 0,
      "failed": 0
    },
    "_clusters": {
      "total": 2,
      "successful": 0,
      "skipped": 0
    },
    "hits": {
      "total": {
        "value": 0,
        "relation": "gte"
      },
      "max_score": null,
      "hits": []
    }
  }
}

// intmd (while search still running)
GET /_async_search/status/FlRYVE81OWZDUm8tMmN3RjQtNEw3SEEbTXCOEliLUdSdzZSZmZkWW15ZmdNQToyMjMz
{
  "id": "FlRYVE81OWZDUm8tMmN3RjQtNEw3SEEbTXCOEliLUdSdzZSZmZkWW15ZmdNQToyMjMz",
  "is_running": true,
  "is_partial": true,
  "start_time_in_millis": 1685726220696,
  "expiration_time_in_millis": 168615822069  ,
  "_shards": {
    "total": 0,
    "successful": 0,
    "skipped": 0,
    "failed": 0
  },
  "_clusters": {
    "total": 2,
    "successful": 0,
    "skipped": 0
  }
}

// final
GET /_async_search/status/FlRYVE81OWZDUm8tMmN3RjQtNEw3SEEbTXCOEliLUdSdzZSZmZkWW15ZmdNQToyMjMz
{
  "id": "FlRYVE81OWZDUm8tMmN3RjQtNEw3SEEbTXCOEliLUdSdzZSZmZkWW15ZmdNQToyMjMz",
  "is_running": false,
  "is_partial": false,
  "start_time_in_millis": 1685726121946,
  "expiration_time_in_millis": 168615812194  ,
  "_shards": {
    "total": 6,
    "successful": 6,
    "skipped": 0,
    "failed": 0
  },
  "_clusters": {
    "total": 2,
    "successful": 2,
    "skipped": 0
  },
  "completion_status": 200
}

The same is true for ccs_minimize_roundtrips=false except there the _cluster object starts in its final state, as normal.

Copy link
Member

@javanna javanna left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

left a couple of questions

@quux00 quux00 force-pushed the ccs/clusters-in-async-search-status branch from 4a75bd5 to 3fe8951 Compare June 5, 2023 15:05
@quux00 quux00 force-pushed the ccs/clusters-in-async-search-status branch 4 times, most recently from 90dfa42 to a0c1cee Compare June 6, 2023 18:23
Copy link
Member

@javanna javanna left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

left a question, LGTM otherwise

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

just checking that you don't forget to remove this comment ;)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Right, sorry. Removing it now. Thanks for the reminder :-)

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This needs to be nullable because older nodes won't send it over the wire. I am less clear on having it null for local-only searches. Shouldn't we use EMPTY in this case like we normally do?

Copy link
Contributor Author

@quux00 quux00 Jun 8, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Currently (before this ticket) _clusters is not shown for local-only searches in the async-status/status response, so that has been the default already.

And in addition, I'm not sure why we use EMPTY for the normal non-CCS case, as that seems wrong to me.

This type of output would be confusing to end users:

{
  "id": "FnhKR2FES241UmNtSjhGWUVTYkhGWWcaemJJaENTNW9SZFdaWnNmRUNzbE1VUTozNjU",
  "is_running": false,
  "is_partial": false,
  "start_time_in_millis": 1686227754043,
  "expiration_time_in_millis": 1686659754043,
  "_shards": {
    "total": 3,
    "successful": 3,
    "skipped": 0,
    "failed": 0
  },
  "_clusters": {
    "total": 0,  // why is this zero?
    "successful": 0, // didn't my local cluster search work?
    "skipped": 0
  },
  "completion_status": 200
}

IMO, the standard non-CCS default Clusters object should be Clusters{total=1, successful=1, skipped=0}. Using empty implies that we don't want to count the local cluster as a cluster. But if that's true, then should we also not count it for CCS? But then that would be also confusing to end users.

If we add EMPTY here for the local-only, we could choose to not serialize it, but that requires extra logic in the code above or if we serialize it but don't want to show it, then that requires extra logic in the toXContent methods. So I chose the path of least resistance which is to just have it be null, which matches current functionality for local-only searches.

If you want EMPTY serialized, then I would say we need to add extra logic in toXContent to not output it to the REST response.

quux00 added 4 commits June 8, 2023 08:54
…sters has a non-zero count total.

When CCS is done, the _cluster object will have a non-zero `total` count and thus
is useful status information for the user.

When a local cluster only async search is done, the _cluster object will not be present in the AsyncStatusResponse.
…bject so it can be obtained

by get-async-search-status coordinators that are not the coordinator for the async search itself.
This requires a Transport Version bump.
…nse when looking to set completion status.

Added several new unit tests in AsyncStatusResponseTests
@quux00 quux00 force-pushed the ccs/clusters-in-async-search-status branch from a0c1cee to 6ad8289 Compare June 8, 2023 12:54
@quux00 quux00 merged commit f032fae into elastic:main Jun 8, 2023
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

>non-issue :Search/Search Search-related issues that do not fall into other categories Team:Search Meta label for search team v8.9.0

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants