You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@impala.apache.org by "Sahil Takiar (Code Review)" <ge...@cloudera.org> on 2019/12/11 01:23:45 UTC

[Impala-ASF-CR] IMPALA-9124 [POC][WIP]: Transparently retry queries that fail due to cluster membership changes

Sahil Takiar has uploaded this change for review. ( http://gerrit.cloudera.org:8080/14824


Change subject: IMPALA-9124 [POC][WIP]: Transparently retry queries that fail due to cluster membership changes
......................................................................

IMPALA-9124 [POC][WIP]: Transparently retry queries that fail due to cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes.

The design of this feature is described in the JIRA, but the TL;DR is
that whenever a ClientRequestState receives an error Status for a query,
it checks if the Status is "Retryable". If it is, then it schedules a
retry of the query using a "query retry" threadpool.

This feature requires touching several parts of the query lifecycle /
Coordinator code:

* The feature is configurable ('retry_failed_queries') and is off by
default.

* It modifies Status objects so they can be classified as "Retryable"
errors. The Coordinator then just retries any Status that is marked as
"Retryable". This is modelled using a new field in TStatus called
TStatusProperties.

* Changes the ClientRequestState so that it can take in an existing
TExecRequest (this is required when retrying queries because the
TExecRequest of the failed query is copied and used for the
ClientRequestState of the retried query).

* ClientRequestState::UpdateQueryStatus is modified such that if it
receives a "Retryable" Status, it schedules a retry of the query.

* ClientRequestState::ExecState is extended with three new states:
RETRYING, RETRIED, and UNKNOWN.

* ImpalaServer::RetryQueryFromThreadPool implements the core logic to
actually retry a failed query.

Additional Notes:

* Retries are transparent to the user. This is achieved by registering the
query id of the failed query with the ClientRequestState of the retried
query. This requires modifying the ImpalaServer
client_request_state_map_ so that different query ids can correspond to
the same ClientRequestState.

This patch is based on three currently in progress changes:
* https://gerrit.cloudera.org/#/c/14677/
* https://gerrit.cloudera.org/#/c/14744/
* https://gerrit.cloudera.org/#/c/14755/

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/common/status.cc
M be/src/common/status.h
M be/src/runtime/krpc-data-stream-sender.cc
M be/src/service/client-request-state-map.h
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/control-service.cc
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-http-handler.h
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
A be/src/service/retry-work.h
M be/src/util/container-util.h
M be/src/util/error-util.h
M common/protobuf/common.proto
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M common/thrift/Status.thrift
M common/thrift/generate_error_codes.py
A tests/custom_cluster/test_query_retries.py
24 files changed, 734 insertions(+), 129 deletions(-)



  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/1
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newchange
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 1
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Michael Ho <mi...@gmail.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#5).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes.

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the leaving node, queries are retried), and (2) if a
query fails and as a result, blacklists a node. Both events are
considered cluster membership changes as they affect what nodes a query
will be scheduled on.

Implementation:
* Query retries are driven by a dedicated threadpool
    * ImpalaServer::RetryQueryFromThreadPool implements the core logic to
      actually retry a failed query.
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A query cannot be retried once any results from the original query
  have been fetched, this is to prevent users from seeing incorrect results

Features:
* Retries are transparent to the user
    * This is achieved by adding a mapping from failed query ids to the
      query id of the retried query
    * ImpalaServer uses this mapping in GetClientFacingRequestState
      which is used to differentiate between "client facing" requests
      for a ClientRequestState vs. internal requrets for a CRS
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
    * "Impala Query Status" is a new field in runtime profiles that
      displays the ClientRequestState execution state (which includes
      the RETRYING and RETRIED states)
    * The Impala Web UI will list all retried queries as being in the
      "RETRIED" state
* Retried queries skip all fe/ planning, authorization, etc.
* This feature is configurable ('retry_failed_queries') and is off by
  default

Refactoring:
* Changes the ClientRequestState so that it can take in an existing
  TExecRequest
    * This is required when retrying queries because the
      TExecRequest of the failed query is copied and used for the
      ClientRequestState of the retried query
* ClientRequestState::ExecState is extended with three new states:
  RETRYING, RETRIED, and UNKNOWN.

Testing:
* Added integration tests in test_query_retries.py, these tests
  consistently pass when run locally
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures

TODO:
* There are some failed tests I am working through
* Additional re-factoring / code cleanup
* Lots more documentation

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
M be/src/service/client-request-state-map.h
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
A be/src/service/retry-work.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
A tests/custom_cluster/test_query_retries.py
16 files changed, 811 insertions(+), 166 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/5
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 5
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 11:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/14824/11/be/src/runtime/query-driver.cc
File be/src/runtime/query-driver.cc:

http://gerrit.cloudera.org:8080/#/c/14824/11/be/src/runtime/query-driver.cc@70
PS11, Line 70: shared_ptr<ClientRequestState> QueryDriver::GetClientRequestState(const TUniqueId& query_id) {
> line too long (94 > 90)
Done



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 11
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Thu, 30 Apr 2020 00:06:15 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 25:

(20 comments)

http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/coordinator.cc
File be/src/runtime/coordinator.cc:

http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/coordinator.cc@937
PS24, Line 937: parent_query_dri
> This isn't valid. (and you have similar issues elsewhere)
Done


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h
File be/src/runtime/query-driver.h:

http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h@1
PS24, Line 1: // Licensed to the Apache Software Foundation (ASF) under one
> The distinction isn't that clear, but it might make more sense to put this 
I think client-request-state probably belongs in the runtime folder. I plan to move some of this around in IMPALA-9370. Left a comment to follow up on this in IMPALA-9370.


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h@39
PS24, Line 39: QueryHandle
> I think this name is confusing, esp. since there's already a QueryHandle in
I like referring to this as a handle, since its really just a pointer into other query level objects.
I think some of these re-factoring / re-renmaing decisions should be done in IMPALA-9370 so that we can tackle them all together and ensure that all class names are consistent with one another.


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h@82
PS24, Line 82: /// *Transparent* Query Retries:
> Not a huge deal in this patch since it's still hidden behind a flag and exp
That is a good point. I added some notes here and in the uber-JIRA as a disclaimer. It's a bit hard to change all the places where this is referred to as "transparent query retries" and I think many folks are already using the phrase. So I think at this point it is a matter of setting proper expectations. I didn't explicitly mention the GetRuntimeProfile() changes so we are planning to revise the behavior in IMPALA-9229
Yeah - I've tested this with Hue, and it works as expected. There are a few observability improvements that could be made to Hue to improve this. The uber-JIRA (IMPALA-9124) has some sub-tasks related to impala-shell usability improvements.


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h@136
PS24, Line 136:   /// query option 'retry_failed_queries') and if the query can be retried. Queries can
> The way you're both returning a Status and also returning some status infor
Doesn't return a Status anymore.
The input Status is used to set the error message "Max retry limit was hit" which does show up in the final error message (see TestQueryRetries::test_multiple_retries)


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h@189
PS24, Line 189:  owns this QueryDriver.
> ?
Forgot to remove this. Deleted.


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h@194
PS24, Line 194: 
              :   /// The ClientRequestState
> ?
Forgot to remove this. Deleted.


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h@204
PS24, Line 204:   /// 'RunFrontendPlanner'.
> Now that this is just move()-ed from 'exec_request_' I'm not sure why its s
Do you mean that CreateRetriedClientRequestState could just pass in a pointer to exec_request_ rather than retry_exec_request_ when creating the new ClientRequestState? I guess we could do that, although I think its a little easier to understand the code in its current form. you move the TExecRequest out from the original ClientRequestState into the new one.


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.cc
File be/src/runtime/query-driver.cc:

http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.cc@97
PS24, Line 97:     if (client_request_state->fetched_rows()) {
> This function would be more readable if you turned these into:
Done


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.cc@302
PS24, Line 302: try_request
> Instead of this make_unique/move() thing, why not just have SetOriginalId()
Done


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-hs2-server.cc
File be/src/service/impala-hs2-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-hs2-server.cc@884
PS24, Line 884:   QueryHandle query_handle;
> I'm not sure that in its current state QueryHandle is really doing much. Th
I think the point of the QueryHandle is to enforce that whenever users use a ClientRequestState, that the shared_ptr to the corresponding QueryDriver is in scope. QueryHandle might not be the perfect way to do this, but I think its better than the previous solution where you had to make sure the shared_ptr<QueryDriver> was always on the stack whenever you used a ClientRequestState. Now you can just use the QueryHandle and as long as it is still in scope, you can use its member variables.
I think the guarantee QueryHandle makes, is that as long as QueryHandle is in scope, you can use its member variables (e.g. ClientRequestState), which feels pretty similar to other object ownership (e.g. an object owns some number of member variables).
I created a GetQueryHandle and a GetActiveQueryHandle as you suggested in your other comments and migrated almost all of the usages of Get.*ClientRequestState to use these methods. The only exception is a few methods in the ImpalaHttpHandler which use the GetQueryDriver::DoFuncForAllEntries method. I probably could re-factor that as well, but not sure it is worth the effort. Open to doing it though if necessary.


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-http-handler.cc
File be/src/service/impala-http-handler.cc:

http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-http-handler.cc@809
PS24, Line 809:   QueryHandle query_handle;
> Noted elsewhere, but I think if you added an 'active' flag to GetQueryHandl
Done. Added a GetActiveQueryHandle().


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-server.h
File be/src/service/impala-server.h:

http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-server.h@932
PS24, Line 932:   /// ClientRequestStates. See 'GetQueryDriver' for a description of the
> Might be worth naming this GetActiveQueryHandle() for clarity.
Done


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-server.cc
File be/src/service/impala-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-server.cc@1160
PS24, Line 1160:   RETURN_IF_ERROR(query_handle.query_driver->StartUnregister());
> I'm not sure why this is necessary. I guess it has the effect of making Unr
This was based on https://gerrit.cloudera.org/#/c/15821/10 - basically there used to be a version of IMPALA-9380 that was written on top of this patch.
I'm not sure I completely follow your comments, but my understanding is that this is needed to support async query unregistration (
IMPALA-9380 - the commit message has some details about thread safety).


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-server.cc@1167
PS24, Line 1167:   unreg_thread_pool_->Offer(move(query_handle));
> I think you'll want to still call move() in situations like this - otherwis
Done


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-server.cc@1586
PS24, Line 1586:     // cluster. CancellationWorkCause::BACKEND_FAILED indicates that a backend running
> I'm wondering why we unregister the query in this case, but only cancel it 
This got deleted since TryQueryRetry doesn't return a Status anymore.


http://gerrit.cloudera.org:8080/#/c/14824/25/be/src/service/impala-server.cc
File be/src/service/impala-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/25/be/src/service/impala-server.cc@642
PS25, Line 642:     Status status = GetQueryHandle(query_id, &query_handle, /*return_unregistered=*/ true);
> line too long (91 > 90)
Done


http://gerrit.cloudera.org:8080/#/c/14824/25/be/src/service/impala-server.cc@644
PS25, Line 644:       ClientRequestState* request_state = query_handle.request_state; 
> line has trailing whitespace
Done


http://gerrit.cloudera.org:8080/#/c/14824/25/be/src/service/impala-server.cc@710
PS25, Line 710:     Status status = GetQueryHandle(query_id, &query_handle, /*return_unregistered=*/ true);
> line too long (91 > 90)
Done


http://gerrit.cloudera.org:8080/#/c/14824/25/be/src/service/impala-server.cc@712
PS25, Line 712:       ClientRequestState* request_state = query_handle.request_state; 
> line has trailing whitespace
Done



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 25
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 08 May 2020 19:16:53 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 9:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/5664/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 9
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Tue, 31 Mar 2020 18:41:52 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 32: Code-Review+2

Carrying +2.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 32
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 15 May 2020 20:10:59 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 4:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/5516/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 4
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 24 Jan 2020 23:16:59 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Thomas Tauber-Marshall (Code Review)" <ge...@cloudera.org>.
Thomas Tauber-Marshall has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 9:

(8 comments)

Still going through this, but some high level thoughts (and a few nit-picks I happened to notice already)

http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/runtime/coordinator.cc
File be/src/runtime/coordinator.cc:

http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/runtime/coordinator.cc@922
PS9, Line 922:           parent_query_driver_->TryQueryRetry(parent_request_state_, &retryable_status));
So obviously the circular dependency here between QueryDriver, ClientRequestState, and Coordinator is unfortunate.

It probably works as is, and it might be difficult to get rid of without significant code restructuring, but I'm concerned its confusing/brittle (eg. this call results in us taking ClientRequestState::lock_ on the ReportExecStatus rpc thread) and so I've been thinking through the options.

Would it be possible to instead have the QueryDriver wait on the coordinator to finish and then check its status and decide whether to retry then?

One problem is the QueryDriver needs to know not just if the query hit an error but if the error was something retryable, but we could do something like have the coordinator remember any nodes it blacklists and expose that info to the QueryDriver.

Another issue is that it means we won't start the retry quite as quickly, since we have to wait for the QueryDriver to notice the error, but that might be fine - its already the case that Coordinator::Wait() will return immediately after an error is reported, without waiting for other backends to be cancelled, see HandleExecStateTransition()->CancelBackends()->backend_exec_complete_barrier_->NotifyRemaining(). The worse case is when the blacklisting info and the error status don't arrive in the same report, eg. the call to TryQueryRetry below, but that should be rare since the error status will have been generated at the same time as the AuxErrorInfo, so you just have to get pretty unlucky with the timing of when the report is generated.

Maybe the bigger issue is I'm not sure its easy to do with the way the code is set up. I was trying to trace through the various Wait()/WaitAsync()/BlockOnWait() whatever calls but I'm not sure how that all works now.


http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/runtime/coordinator.cc@1060
PS9, Line 1060:   ExecEnv::GetInstance()->cluster_membership_mgr()->BlacklistExecutor(
This of course doesn't actually guarantee that the retried query won't be scheduled on the executor that gets blacklisted, eg. if it takes longer for the query to get through admission control again than the blacklist timeout.

We might want to consider doing something like passing through a list of executors to reschedule on, to avoid having queries repeatedly fail in the same way. On the other hand, the blacklist timeout was designed to be longer than it should take the statestore to notice the executor is down, so in theory if the executor really is down maybe it doesn't matter.

Probably not necessary to do anything different for this patch, though, just wanted to point this out.


http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/runtime/query-driver.h
File be/src/runtime/query-driver.h:

http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/runtime/query-driver.h@113
PS9, Line 113: (2) the
             :   /// query has already been retried
not sure what this is supposed to mean


http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/service/client-request-state.h
File be/src/service/client-request-state.h:

http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/service/client-request-state.h@84
PS9, Line 84: UNKNOWN
I don't think this is used anywhere?


http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/service/impala-hs2-server.cc
File be/src/service/impala-hs2-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/service/impala-hs2-server.cc@148
PS9, Line 148:   unique_ptr<TExecRequest> exec_request = make_unique<TExecRequest>();
I don't think this is used anywhere?


http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/service/impala-hs2-server.cc@763
PS9, Line 763:   ClientRequestState* request_state = nullptr;
So I'm wondering if its safe to no longer have any shared_ptr here and in the other places in this file - what's to stop the QueryDriver from getting deleted by Unregister while this function is executing, which would make this pointer no longer valid?


http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/service/impala-hs2-server.cc@1039
PS9, Line 1039:   // If the query was retried, fetch the profile for the most recent attempt of the query
I think its definitely necessary that we provide a way for all relevant profiles to be accessed through HS2, not just the webserver.

This is one case where the retries basically just can't be "transparent", eg. in your current solution clients get the confusing behavior of requesting a profile for a particular query_id and getting back a profile with a different query_id listed, and we should think carefully about the right behavior to make it as easy as possible for clients.

Some options:
- Return the profile corresponding to the provided query_id. Clients would need to follow the "Retried Query Id" in the profile to get the profile of the retry.
- Return all profiles for all retries together in a single call, possibly requiring a flag like "get_all_retries" to be set on the request.


http://gerrit.cloudera.org:8080/#/c/14824/9/common/thrift/ImpalaService.thrift
File common/thrift/ImpalaService.thrift:

http://gerrit.cloudera.org:8080/#/c/14824/9/common/thrift/ImpalaService.thrift@523
PS9, Line 523: branch
typo



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 9
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Tue, 07 Apr 2020 19:58:08 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 23:

Fixed compilation issue. Rebased against and resolved some conflicts on top of IMPALA-9692.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 23
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Thu, 07 May 2020 00:54:35 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 27:

(25 comments)

> I know I've asked for a lot, but I think the patch is a lot better now.

Not a problem, thanks for all the thorough feedback. Don't think this patch would be nearly as clean without all the comments.

> I still have a lot of concerns about this feature, esp. about our story around observability, but I think those concerns can mostly be addressed in follow up work and most of my remaining comments are more minor things.

Right. I don't think the expectation should be that this feature is complete by any means. I'm not sure how feature development has been done historically in Impala, but IMO I think features need to be built piece-meal over multiple-commits, before they can really be used. I won't rant too long about this, but I just think that getting an initial version that users can at least play around with gives us a level of feedback and testing that can't be done within a scoped design review session / single code review.

http://gerrit.cloudera.org:8080/#/c/14824/26/be/src/runtime/coordinator.cc
File be/src/runtime/coordinator.cc:

http://gerrit.cloudera.org:8080/#/c/14824/26/be/src/runtime/coordinator.cc@1079
PS26, Line 1079: 
> I don't think that's true - ClientRequestState::UpdateQueryStatus() is never called from inside Coordinator.

I guess, a '-->' wasn't the correct way to describe this. StartBackendExec() returns an error to Exec() which returns an error to FinishExecQueryOrDmlRequest() which then calls UpdateQueryStatus().

> Now, MarkAsRetrying() will set an error on ClientRequestState but after that Coordinator::Exec() will still return an error which ClientRequestState will try to set on itself with UpdateQueryStatus() but it'll get discarded because an error was already set.

Right.

> The point is, its messy and I'm concerned it will be error prone.

I agree that it is more complex. I'm not sure I understand why it is particularly more error prone.

It's not that I don't want to take your suggestion, I just think at this point it is better to address this separately. My hesitation is that doing this re-factoring now is going to introduce bugs into the patch unnecessarily. From experience when first iterating on this patch, doing seemingly harmless re-factoring has introduced a lot of bugs. So I'm not really convinced it is worth the effort at this point in time.

Taking a step back, I think at this point in the feature lifecyle, its not worth investing a huge amount of time on this. The feature itself it still immature, and subject to change. It's hard to predict exactly what it will look like after all the planned enhancement, so a lot of this debate might end up being a moot point.

I am taking note of this feedback though, and plan to track it in IMPALA-9370.


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/runtime/coordinator.cc
File be/src/runtime/coordinator.cc:

http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/runtime/coordinator.cc@910
PS27, Line 910:   Status status = Status::OK();
> I don't think declaring this here is necessary since its not used outside o
Done


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/runtime/query-driver.h
File be/src/runtime/query-driver.h:

http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/runtime/query-driver.h@58
PS27, Line 58: std::shared_ptr<QueryDriver>
> I think you might want to make this a '&' to avoid unnecessary copies
isn't the point here that the pointer returned by query_driver() should be able to out-live the QueryHandle instance? in which case you would need to take a shared_ptr copy to ensure proper ref counting?

at least, I thought that was part of the argument for creating the whole "->" and "*" operator overloading.


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/runtime/query-driver.h@93
PS27, Line 93: TryRetryQuery
> TryQueryRetry
Done


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h
File be/src/runtime/query-driver.h:

http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h@136
PS24, Line 136: ///
> Right, but MarkAsRetrying doesn't get called in the case where you call AddDetail() because TryQueryRetry returns immediately after AddDetail()

Whoops, yeah you are right. I think in that case the error still gets propagated to the the ClientRequestState::query_status_ though, which is why the errors show up in the runtime profile.


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/runtime/query-driver.cc
File be/src/runtime/query-driver.cc:

http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/runtime/query-driver.cc@228
PS27, Line 228:   // Run the new query.
> I think its fine to leave for now, but just wanted to note that the duplica
Yeah - fair point. I tried to do that as much as I could. Most of the calls below are just one off calls to the ClientRequestState - e.g. there isn't that much control flow (or at least I tried to minimize the amount of control flow logic).

The other big difference is that the error handling here vs. elsewhere is very different.


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/client-request-state.h
File be/src/service/client-request-state.h:

http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/client-request-state.h@214
PS27, Line 214:   Status InitExecRequest(const TQueryCtx& query_ctx);
> This is no longer used anywhere
Thanks for catching that.


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/client-request-state.h@222
PS27, Line 222:   std::string ExecStateToString(ExecState state) const;
> static?
Done


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/client-request-state.h@225
PS27, Line 225:   std::string RetryStateToString(RetryState state) const;
> static?
Done


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/client-request-state.h@228
PS27, Line 228:   std::shared_ptr<ImpalaServer::SessionState> session() const { return session_; }
> I think this copies the shared_ptr, which adds another atomic inc/dec. Is t
I think the session can get expired / deleted in various places in ImpalaServer - e.g. an explicit call to close the session, or the session timeout thread. So I think the shared_ptr is necessary for async deletion of the session. At least, that seems to be the pattern with other usages of the session.

I re-factored TryQueryRetry a bit so that it only calls session() once, which should decrease the number of times it has to acquire the shared_ptr.


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/client-request-state.h@556
PS27, Line 556: Updated by
              :   /// UpdateQueryStatus
> No longer true, due to MarkAsRetrying
Done


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/client-request-state.h@561
PS27, Line 561: initialized in InitExecRequest
> No longer true. Also maybe note that the QueryDriver owns this.
Done


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-beeswax-server.cc
File be/src/service/impala-beeswax-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-beeswax-server.cc@420
PS27, Line 420: Don't use
              :   // GetActiveQueryHandle because clients can request profiles for unregistered queries.
> Not sure what this is supposed to mean
Deleted, I think that comment was from before GetActiveQueryHandle existed, it probably used to be GetQueryHandle and :%s/GetQueryHandle/GetActiveQueryHandle/g inadvertently changed it.


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-server.h
File be/src/service/impala-server.h:

http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-server.h@670
PS27, Line 670: query_driver
> query_handle
Done


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-server.h@868
PS27, Line 868:     /// query_state = union(beeswax::QueryState, ClientRequestState::RetryState).
> Probably requires more explanation, eg. "used for the http server..."
Done


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-server.h@985
PS27, Line 985: 'request_state'
> needs to be updated
Changed to the more vague 'exec state', which is consistent with the phrasing of the Beeswax version of this method.


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-server.h@1024
PS27, Line 1024: ClientRequestState
> needs to be updated
Done


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-server.cc
File be/src/service/impala-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-server.cc@1160
PS24, Line 1160:   // unregistration work. StartUnregister() succeeds for the first thread to call it to
> Ah okay, I hadn't noticed the equivalent started_finalize thing in ClientRe
Done


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-server.cc
File be/src/service/impala-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-server.cc@943
PS27, Line 943:   (*query_handle).EmplaceQueryDriver(this);
> Similar to noted below, you could clean this up by adding a static QueryDri
Responded in other comment.


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-server.cc@1177
PS27, Line 1177: &*
> I think you could just pass a const QueryHandle& here and it would be clean
CloseClientRequestState requires a bunch of non-const access - e.g. it passes the CRS into GetExecSummary which needs to call request_state->summary_profile()->SetTExecSummary(t_exec_summary); (which is a non-const function).


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-server.cc@1297
PS27, Line 1297: shared_ptr<QueryDriver>
> So I believe this copies the shared_ptr, which results in an extra atomic i
Not sure I fully follow your suggestion. I made the following changes, which I think make things a bit cleaner.

Added the method SetHandle(...) in QueryHandle, which sets both the QueryDriver and CRS fields.

Added the static method CreateQueryDriver(...) in QueryDriver.

I made QueryDriver a friend of QueryHandle, but I couldn't get rid of EmplaceQueryDriver. C++ wouldn't let me use private member variables of QueryHandle inside a static method in QueryDriver.


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-server.cc@1504
PS27, Line 1504: QueryHandle handle;
> I think this is unnecessary and adds another shared_ptr copy. You can just 
Done


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-server.cc@2411
PS27, Line 2411: query_handle
> request_state?
Done


http://gerrit.cloudera.org:8080/#/c/14824/27/tests/custom_cluster/test_query_retries.py
File tests/custom_cluster/test_query_retries.py:

http://gerrit.cloudera.org:8080/#/c/14824/27/tests/custom_cluster/test_query_retries.py@92
PS27, Line 92:   def test_kill_impalad_expect_retry(self):
> I don't think I see a test case in here for if the retry is attempted due t
This was sort of implicitly tested in some other tests, but I added an explicit test for Exec() RPCs, and add some additional validation for that specific case.


http://gerrit.cloudera.org:8080/#/c/14824/27/tests/custom_cluster/test_query_retries.py@363
PS27, Line 363:   def test_retry_query_hs2(self):
> Fwiw, since beeswax is now deprecated and pretty much all clients use hs2, 
Yeah, I thought about this while writing these tests. My goal was to make sure that all code paths are covered, which I think this test suite is still achieving.

Most of the query retry logic is agnostic to the specific protocol used, so I think whether it is beeswax or hs2, most of the time it shouldn't make a big difference, because the code paths covered will be the same.

I think this test should cover all the code paths that are unique to HS2.



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 27
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Tue, 12 May 2020 23:01:40 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Joe McDonnell, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#31).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes (IMPALA-9124).

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the removed node, queries are retried), or (2) if a
query fails and as a result, blacklists a node. Either event is
considered a cluster membership change as it affects what nodes a query
will be scheduled on. The assumption is that a retry of the query with
the updated cluster membership will succeed.

A query retry is modelled as a brand new query, with its own query id.
This simplifies the implementation and the resulting runtime profiles
when queries are retried.

Core Features:
* Retries are transparent to the user; no modification to client
  libraries are necessary to support query retries
* Retried queries skip all fe/ parsing, planning, authorization, etc.
* Retries are configurable ('retry_failed_queries') and are off by
  default

Implementation:
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A new layer of abstraction between the ImpalaServer and
  ClientRequestState has been added; it is called the QueryDriver
* Each ClientRequestState is treated as a single attempt of a query, and
  the QueryDriver owns all ClientRequestStates for a query
* ClientRequestState has a new state object called RetryState; a
  ClientRequestState can either be NOT_RETRIED, RETRYING, or RETRIED
* The QueryDriver owns the TExecRequest for the query as well, it is
  re-used for each query retry
* QueryDrivers and ClientRequestStates are now referenced using a
  QueryHandle

Observability:
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
* Runtime profiles of queries that fail and then are retried will have:
    * "Retry Status: RETRIED"
    * "Retry Cause: [the error that triggered the retry]"
    * "Retried Query Id: [the query id of the retried query]"
* Runtime profiles of the retried query (e.g. the second attempt of the
  query) will include:
    * "Original Query Id: [the query id of the original query]"
* The Impala Web UI will list all retried queries as being in the
  "RETRIED" state

Testing:
* Added E2E tests in test_query_retries.py; looped tests for a few days
* Added a stress test query_retries_stress_runner.py that runs concurrent
  streams of a TPC-{H,DS} workload and randomly kills impalads
* Ran the stress test with various configurations: tpch on parquet,
  tpcds on parquet, tpch 30 GB on parquet (one stream), tpcds 30 GB on
  parquet (one stream), tpch on text, tpcds on text
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures
* Ran 30 GB TPC-DS workload on a 3 node cluster, randomly restarted
  impalads, and manually verified that queries were retried
* Manually tested retries work with various clients, specifically the
  impala-shell and Hue
* Ran core tests and query retry stress test against an ASAN build
* Ran concurrent_select.py to stress query cancellation
* Ran be/ tests against a TSAN build

Limitations:
* There are several limitations that are listed out in the parent JIRA

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/benchmarks/process-wide-locks-benchmark.cc
M be/src/runtime/CMakeLists.txt
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
A be/src/runtime/query-driver.cc
A be/src/runtime/query-driver.h
M be/src/service/CMakeLists.txt
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/control-service.cc
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
R be/src/service/query-driver-map.cc
A be/src/service/query-driver-map.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/testutil/impalad-query-executor.cc
M be/src/testutil/impalad-query-executor.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M tests/common/impala_cluster.py
M tests/common/impala_service.py
A tests/custom_cluster/test_query_retries.py
A tests/stress/query_retries_stress_runner.py
27 files changed, 2,700 insertions(+), 696 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/31
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 31
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 24:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/5990/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 24
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Thu, 07 May 2020 15:50:54 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 31: Verified-1

Build failed: https://jenkins.impala.io/job/gerrit-verify-dryrun/5839/


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 31
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 15 May 2020 08:27:59 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 30: Verified-1

Build failed: https://jenkins.impala.io/job/gerrit-verify-dryrun/5836/


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 30
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 15 May 2020 01:22:42 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Joe McDonnell, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#24).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes (IMPALA-9124).

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the removed node, queries are retried), or (2) if a
query fails and as a result, blacklists a node. Either event is
considered a cluster membership change as it affects what nodes a query
will be scheduled on. The assumption is that a retry of the query with
the updated cluster membership will succeed.

A query retry is modelled as a brand new query, with its own query id.
This simplifies the implementation and the resulting runtime profiles
when queries are retried.

Core Features:
* Retries are transparent to the user; no modification to client
  libraries are necessary to support query retries
* Retried queries skip all fe/ parsing, planning, authorization, etc.
* Retries are configurable ('retry_failed_queries') and are off by
  default

Implementation:
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A new layer of abstraction between the ImpalaServer and
  ClientRequestState has been added; it is called the QueryDriver
* Each ClientRequestState is treated as a single attempt of a query, and
  the QueryDriver owns all ClientRequestStates for a query
* ClientRequestState has a new state object called RetryState; a
  ClientRequestState can either be NOT_RETRIED, RETRYING, or RETRIED
* The QueryDriver owns the TExecRequest for the query as well, it is
  re-used for each query retry

Observability:
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
* Runtime profiles of queries that fail and then are retried will have:
    * "Retry Status: RETRIED"
    * "Retry Cause: [the error that triggered the retry]"
    * "Retried Query Id: [the query id of the retried query]"
* Runtime profiles of the retried query (e.g. the second attempt of the
  query) will include:
    * "Original Query Id: [the query id of the original query]"
* The Impala Web UI will list all retried queries as being in the
  "RETRIED" state

Testing:
* Added E2E tests in test_query_retries.py; looped tests for a few days
* Added a stress test query_retries_stress_runner.py that runs concurrent
  streams of a TPC-{H,DS} workload and randomly kills impalads
* Ran the stress test with various configurations: tpch on parquet,
  tpcds on parquet, tpch 30 GB on parquet (one stream), tpcds 30 GB on
  parquet (one stream), tpch on text, tpcds on text
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures
* Ran 30 GB TPC-DS workload on a 3 node cluster, randomly restarted
  impalads, and manually verified that queries were retried
* Manually tested retries work with various clients, specifically the
  impala-shell and Hue
* Ran core tests and query retry stress test against an ASAN build
* Ran concurrent_select.py to stress query cancellation
* Ran be/ tests against a TSAN build, filed IMPALA-9730 as a follow up

Limitations:
* There are several limitations that are listed out in the parent JIRA

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/benchmarks/process-wide-locks-benchmark.cc
M be/src/runtime/CMakeLists.txt
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
A be/src/runtime/query-driver.cc
A be/src/runtime/query-driver.h
M be/src/service/CMakeLists.txt
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/control-service.cc
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
R be/src/service/query-driver-map.cc
A be/src/service/query-driver-map.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/testutil/impalad-query-executor.cc
M be/src/testutil/impalad-query-executor.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M common/thrift/generate_error_codes.py
M tests/common/impala_cluster.py
M tests/common/impala_service.py
A tests/custom_cluster/test_query_retries.py
A tests/stress/query_retries_stress_runner.py
28 files changed, 2,458 insertions(+), 510 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/24
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 24
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 17:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/14824/17/be/src/service/impala-server.h
File be/src/service/impala-server.h:

http://gerrit.cloudera.org:8080/#/c/14824/17/be/src/service/impala-server.h@249
PS17, Line 249:   virtual void Cancel(impala::TStatus& status, const beeswax::QueryHandle& beeswax_handle);
line too long (91 > 90)



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 17
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Wed, 06 May 2020 01:24:24 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 23:

Build Failed 

https://jenkins.impala.io/job/gerrit-code-review-checks/5983/ : Initial code review checks failed. See linked job for details on the failure.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 23
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Thu, 07 May 2020 01:06:30 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9124 [POC][WIP]: Transparently retry queries that fail due to cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9124 [POC][WIP]: Transparently retry queries that fail due to cluster membership changes
......................................................................


Patch Set 1:

Whoops didn't mean to publish this.

I split the changes to Status into a separate patch to make things easier to review: https://gerrit.cloudera.org/#/c/14882/


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 1
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Michael Ho <mi...@gmail.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Wed, 11 Dec 2019 01:24:29 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 14:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/14824/14/be/src/runtime/query-driver.cc
File be/src/runtime/query-driver.cc:

http://gerrit.cloudera.org:8080/#/c/14824/14/be/src/runtime/query-driver.cc@163
PS14, Line 163:       &QueryDriver::RetryQueryFromThread, this, error, query_driver, &retry_query_thread_));
> line too long (92 > 90)
Done



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 14
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 01 May 2020 21:34:45 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Thomas Tauber-Marshall (Code Review)" <ge...@cloudera.org>.
Thomas Tauber-Marshall has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 5:

(7 comments)

http://gerrit.cloudera.org:8080/#/c/14824/5//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/14824/5//COMMIT_MSG@26
PS5, Line 26: A query cannot be retried once any results from the original query
            :   have been fetched, this is to prevent users from seeing incorrect results
Do we have any mechanism for a user to specify that they want their results to always be spooled until the query is complete to ensure that any failed queries that are retriable do in fact get retried?


http://gerrit.cloudera.org:8080/#/c/14824/5//COMMIT_MSG@41
PS5, Line 41: The Impala Web UI will list all retried queries as being in the
            :       "RETRIED" state
This may be a reasonable approach, but I think we should think about the user experience implications of this.

In particular, it means that a retried query will show up as two separate queries on the webui, which is potentially confusing, esp. since the retry will have a different query_id that won't correspond the anything user-facing (eg. won't be the query_id printed out by impala-shell) and as you've got the patch right now, I'm not even sure if there's a way to figure out which retries would correspond to which original queries. We could of course do something like add the retry query id to the profile of the original query, but then people have to click through to the profile to match things up.

There's also the problem that once CloseClientRequestState is called at the end of RetryQueryFromThreadPool, the original query will show up in the "Completed Queries" section (and may even disappear from there before the retry query completes if the query_log fills up), which is again probably confusing since from the perspective of the client that query is really still running.

Some of this could possibly wait until a follow up patch to work out, as long as we've got a plan.


http://gerrit.cloudera.org:8080/#/c/14824/5/be/src/service/client-request-state-map.h
File be/src/service/client-request-state-map.h:

http://gerrit.cloudera.org:8080/#/c/14824/5/be/src/service/client-request-state-map.h@38
PS5, Line 38: bool overwrite = false
I don't think this is being used anywhere anymore.


http://gerrit.cloudera.org:8080/#/c/14824/5/be/src/service/client-request-state.h
File be/src/service/client-request-state.h:

http://gerrit.cloudera.org:8080/#/c/14824/5/be/src/service/client-request-state.h@82
PS5, Line 82: RETRYING, RETRIED, UNKNOWN
I think this approach, and the way you have to save the previous state and then sometimes use this or sometimes the previous state, is confusing.

Its probably more straight forward to just have a separate 'enum Retry State { RETRYING, RETRIED, NOT_RETRIED }

From my other comments, if we add a ClientRequestState wrapper, it could go in there, or even might not be necessary since the wrapper would already know if we're retrying based on if there's multiple ClientRequestStates that its wrapped around.


http://gerrit.cloudera.org:8080/#/c/14824/5/be/src/service/impala-server.cc
File be/src/service/impala-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/5/be/src/service/impala-server.cc@632
PS5, Line 632:     shared_ptr<ClientRequestState> request_state = GetClientFacingRequestState(query_id);
Unless I'm missing something in this patch, this means that only the profile for the retry query will be available through the GetRuntimeProfile() api. Possibly fine to leave for followup work, but I think we definitely need to have some way of returning the profile for the original query too.


http://gerrit.cloudera.org:8080/#/c/14824/5/be/src/service/impala-server.cc@920
PS5, Line 920:   unique_ptr<TExecRequest> exec_request = make_unique<TExecRequest>();
So I'm wondering if this patch would benefit from adding another layer of abstraction.

What I have in mind is something like a wrapper around ClientRequestState that can contain multiple ClientRequestStates (though possibly we would name the wrapper ClientRequestState and rename the current ClientRequestState to something like QueryInstanceState or whatever).

Some benefits of doing this:
- It would provide a natural place for TExecRequest to live without doing this stuff with unique_ptr and std::move, as well as anything else that's common across all retries.
- It would allow us to hide some of the details of retries from impala-server, eg. we wouldn't need the logic around BlockOnWait that you've duplicated between the hs2 server and the beeswax server.


http://gerrit.cloudera.org:8080/#/c/14824/5/be/src/service/impala-server.cc@1025
PS5, Line 1025: query_ctx->query_id = UuidToQueryId(random_generator()());
As mentioned before, I definitely think it would be awesome if we could create a separation between "user facing query_id" and "internal query_id".

It would solve some user experisnce problems, for example right now if someone is debugging issues and looking through the logs, they're going to have to manually figure out the mapping from original query_id to retry query_id from looking at the profile or something (or I guess we can log the mapping). Similarly, the issues I've mentioned elsewhere with the webui. And in general its a cleaner abstraction than the current "the first query_id becomes the user-facing query_id, and the subsequent retry ids are treated differently".

Its also potentially a lot more work, since we use query_id all over the place and we'd have to think about the places where we want to use "internal" vs. "user facing" id and pass both through in a lot of places.

Maybe a compromise would be to do something clever like have all of the query_ids for a query and its retries share the same 'hi' with different 'lo's to make it obvious which queries correspond to each other.



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 5
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 31 Jan 2020 21:12:11 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Joe McDonnell, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#26).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes (IMPALA-9124).

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the removed node, queries are retried), or (2) if a
query fails and as a result, blacklists a node. Either event is
considered a cluster membership change as it affects what nodes a query
will be scheduled on. The assumption is that a retry of the query with
the updated cluster membership will succeed.

A query retry is modelled as a brand new query, with its own query id.
This simplifies the implementation and the resulting runtime profiles
when queries are retried.

Core Features:
* Retries are transparent to the user; no modification to client
  libraries are necessary to support query retries
* Retried queries skip all fe/ parsing, planning, authorization, etc.
* Retries are configurable ('retry_failed_queries') and are off by
  default

Implementation:
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A new layer of abstraction between the ImpalaServer and
  ClientRequestState has been added; it is called the QueryDriver
* Each ClientRequestState is treated as a single attempt of a query, and
  the QueryDriver owns all ClientRequestStates for a query
* ClientRequestState has a new state object called RetryState; a
  ClientRequestState can either be NOT_RETRIED, RETRYING, or RETRIED
* The QueryDriver owns the TExecRequest for the query as well, it is
  re-used for each query retry

Observability:
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
* Runtime profiles of queries that fail and then are retried will have:
    * "Retry Status: RETRIED"
    * "Retry Cause: [the error that triggered the retry]"
    * "Retried Query Id: [the query id of the retried query]"
* Runtime profiles of the retried query (e.g. the second attempt of the
  query) will include:
    * "Original Query Id: [the query id of the original query]"
* The Impala Web UI will list all retried queries as being in the
  "RETRIED" state

Testing:
* Added E2E tests in test_query_retries.py; looped tests for a few days
* Added a stress test query_retries_stress_runner.py that runs concurrent
  streams of a TPC-{H,DS} workload and randomly kills impalads
* Ran the stress test with various configurations: tpch on parquet,
  tpcds on parquet, tpch 30 GB on parquet (one stream), tpcds 30 GB on
  parquet (one stream), tpch on text, tpcds on text
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures
* Ran 30 GB TPC-DS workload on a 3 node cluster, randomly restarted
  impalads, and manually verified that queries were retried
* Manually tested retries work with various clients, specifically the
  impala-shell and Hue
* Ran core tests and query retry stress test against an ASAN build
* Ran concurrent_select.py to stress query cancellation
* Ran be/ tests against a TSAN build, filed IMPALA-9730 as a follow up

Limitations:
* There are several limitations that are listed out in the parent JIRA

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/benchmarks/process-wide-locks-benchmark.cc
M be/src/runtime/CMakeLists.txt
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
A be/src/runtime/query-driver.cc
A be/src/runtime/query-driver.h
M be/src/service/CMakeLists.txt
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/control-service.cc
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
R be/src/service/query-driver-map.cc
A be/src/service/query-driver-map.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/testutil/impalad-query-executor.cc
M be/src/testutil/impalad-query-executor.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M common/thrift/generate_error_codes.py
M tests/common/impala_cluster.py
M tests/common/impala_service.py
A tests/custom_cluster/test_query_retries.py
A tests/stress/query_retries_stress_runner.py
28 files changed, 2,482 insertions(+), 513 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/26
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 26
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Thomas Tauber-Marshall (Code Review)" <ge...@cloudera.org>.
Thomas Tauber-Marshall has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 28: Code-Review+2

(5 comments)

http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/runtime/query-driver.h
File be/src/runtime/query-driver.h:

http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/runtime/query-driver.h@58
PS27, Line 58: inline ClientRequestState* o
> isn't the point here that the pointer returned by query_driver() should be 
Sure, I mean there are basically two cases for how this might be used:
- Most of the uses of this in the current patch are calls like "query_handle.query_driver()->SomeQueryDriverFn()", in which case the QueryHandle is going to be in scope for the whole call and making this a "const&" saves an atomic inc/dec without any risk.
- If a developer wants to make a local variable like "shared_ptr<QueryDriver> = query_handle.query_driver()",  making this a "const&" gives them the option of whether or not to make a copy (i.e. whether the local variable itself is declared a const& or not). This introduces a little risk, cause a dev could not make a copy in a situation where they actually needed one.

Its different than the case with adding the operator-> since there we were returning a bare pointer to a ClientRequestState which doesn't allow the caller to get any guarantees on its lifetime even if they want.

Obviously this isn't really super perf critical code, so its not a big deal. Feel free to leave as is.


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h
File be/src/runtime/query-driver.h:

http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h@136
PS24, Line 136: /// id, which will be distinct from the query id of the originally submitted query that
> I think in that case the error still
 > gets propagated to the the ClientRequestState::query_status_
 > though, which is why the errors show up in the runtime profile.

I think that's true in some cases but not in other cases, but its fine.


http://gerrit.cloudera.org:8080/#/c/14824/28/be/src/runtime/query-driver.cc
File be/src/runtime/query-driver.cc:

http://gerrit.cloudera.org:8080/#/c/14824/28/be/src/runtime/query-driver.cc@35
PS28, Line 35: StartUnregister
Finalize()


http://gerrit.cloudera.org:8080/#/c/14824/28/be/src/service/impala-hs2-server.cc
File be/src/service/impala-hs2-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/28/be/src/service/impala-hs2-server.cc@150
PS28, Line 150:   query_driver->CreateClientRequestState(query_ctx, session, &query_handle);
You could also move the call to CreateClientRequestState into CreateNewDriver and then I think it wouldn't need to be public anymore and you wouldn't need to define the 'query_driver' variable on the line before this, and then pass the 'query_handle' into RegisterQuery which is nice for being consistent about passing QueryHandles everywhere.

Fine to leave as is though


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-server.cc
File be/src/service/impala-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-server.cc@1177
PS27, Line 1177: _m
> CloseClientRequestState requires a bunch of non-const access - e.g. it pass
In that case, a non-const QueryHandle& or a QueryHandle*. Not a big deal though



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 28
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Wed, 13 May 2020 18:01:22 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#7).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes.

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the leaving node, queries are retried), and (2) if a
query fails and as a result, blacklists a node. Both events are
considered cluster membership changes as they affect what nodes a query
will be scheduled on.

Implementation:
* Query retries are driven by a dedicated threadpool
    * ImpalaServer::RetryQueryFromThreadPool implements the core logic to
      actually retry a failed query.
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A query cannot be retried once any results from the original query
  have been fetched, this is to prevent users from seeing incorrect results

Features:
* Retries are transparent to the user
    * This is achieved by adding a mapping from failed query ids to the
      query id of the retried query
    * ImpalaServer uses this mapping in GetClientFacingRequestState
      which is used to differentiate between "client facing" requests
      for a ClientRequestState vs. internal requrets for a CRS
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
    * "Impala Query Status" is a new field in runtime profiles that
      displays the ClientRequestState execution state (which includes
      the RETRYING and RETRIED states)
    * The Impala Web UI will list all retried queries as being in the
      "RETRIED" state
* Retried queries skip all fe/ planning, authorization, etc.
* This feature is configurable ('retry_failed_queries') and is off by
  default

Refactoring:
* Changes the ClientRequestState so that it can take in an existing
  TExecRequest
    * This is required when retrying queries because the
      TExecRequest of the failed query is copied and used for the
      ClientRequestState of the retried query
* ClientRequestState::ExecState is extended with three new states:
  RETRYING, RETRIED, and UNKNOWN.

Testing:
* Added integration tests in test_query_retries.py, these tests
  consistently pass when run locally
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures

TODO:
* Additional re-factoring / code cleanup
* Lots more documentation

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
A be/src/service/retry-work.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M common/thrift/generate_error_codes.py
A tests/custom_cluster/test_query_retries.py
16 files changed, 793 insertions(+), 168 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/7
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 7
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Thomas Tauber-Marshall (Code Review)" <ge...@cloudera.org>.
Thomas Tauber-Marshall has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 27:

(25 comments)

Thanks for the changes. I know I've asked for a lot, but I think the patch is a lot better now.

I still have a lot of concerns about this feature, esp. about our story around observability, but I think those concerns can mostly be addressed in follow up work and most of my remaining comments are more minor things.

http://gerrit.cloudera.org:8080/#/c/14824/26/be/src/runtime/coordinator.cc
File be/src/runtime/coordinator.cc:

http://gerrit.cloudera.org:8080/#/c/14824/26/be/src/runtime/coordinator.cc@1079
PS26, Line 1079: 
> Before this patch, if a query fails: ClientRequestState::FinishExecQueryOrDmlRequest()
 > --> Coordinator::Exec() --> Coordinator::StartBackendExec() -->
 > ClientRequestState::UpdateQueryStatus() which sets
 > ClientRequestState::query_status_.

I don't think that's true - ClientRequestState::UpdateQueryStatus() is never called from inside Coordinator. If an Exec() rpc fails, before this patch Coordinator::Exec() will return an error status and ClientRequestState will call UpdateQueryStatus() on itself to set the error that's returned from Exec().

Now, MarkAsRetrying() will set an error on ClientRequestState but after that Coordinator::Exec() will still return an error which ClientRequestState will try to set on itself with UpdateQueryStatus() but it'll get discarded because an error was already set.

As far as I can tell, prior to this patch if the Coordinator needed to communicate an error to the ClientRequestState it always did so by returning an error Status, eg. if a backend reports an error usually that gets bubbled up when the ClientRequestState calls GetNext(), and now we've added this other patch where the ClientRequestState calls into the Coordinator and during that call the Coordinator calls back into the ClientRequestState to set the error even though its also about to return another Status that represents the same error with a slightly different text. The point is, its messy and I'm concerned it will be error prone.

 > A lot of the re-factoring we end up
 > doing now might just be wasted work if we end up changing half the
 > logic.

Sure, you don't have to take my suggestions, esp. if you have a good reason/a vision of your own for how to clean this all up.

I'm trying to help make sure this patch is well designed because I think it'll make the further work easier, eg. minimizing the amount of retry related logic that ends up in Coordinator should make it easier to make further changes to how retries work.

If you're really determined to keep the calls to TryQueryRetry in Coordinator in this patch, its fine with me.


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/runtime/coordinator.cc
File be/src/runtime/coordinator.cc:

http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/runtime/coordinator.cc@910
PS27, Line 910:   Status status = Status::OK();
I don't think declaring this here is necessary since its not used outside of the if/else immediately below


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/runtime/query-driver.h
File be/src/runtime/query-driver.h:

http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/runtime/query-driver.h@58
PS27, Line 58: std::shared_ptr<QueryDriver>
I think you might want to make this a '&' to avoid unnecessary copies


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/runtime/query-driver.h@93
PS27, Line 93: TryRetryQuery
TryQueryRetry


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h
File be/src/runtime/query-driver.h:

http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h@136
PS24, Line 136: ///
> TryQueryRetry passes the status to
 > MarkAsRetrying which sets the given status as the query_status_ of
 > the ClientRequestState, which is what gets exposed in the runtime
 > profile.

Right, but MarkAsRetrying doesn't get called in the case where you call AddDetail() because TryQueryRetry returns immediately after AddDetail()


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/runtime/query-driver.cc
File be/src/runtime/query-driver.cc:

http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/runtime/query-driver.cc@228
PS27, Line 228:   // Run the new query.
I think its fine to leave for now, but just wanted to note that the duplication of the query execution logic between here and ImpalaServer is unfortunate and it might be nice to have more of a QueryDriver::RunQuery() type function that works for both initial attempts and retries when we're doing refactoring.


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/client-request-state.h
File be/src/service/client-request-state.h:

http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/client-request-state.h@214
PS27, Line 214:   Status InitExecRequest(const TQueryCtx& query_ctx);
This is no longer used anywhere


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/client-request-state.h@222
PS27, Line 222:   std::string ExecStateToString(ExecState state) const;
static?


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/client-request-state.h@225
PS27, Line 225:   std::string RetryStateToString(RetryState state) const;
static?


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/client-request-state.h@228
PS27, Line 228:   std::shared_ptr<ImpalaServer::SessionState> session() const { return session_; }
I think this copies the shared_ptr, which adds another atomic inc/dec. Is there a reason we need to return a shared_ptr here?


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/client-request-state.h@556
PS27, Line 556: Updated by
              :   /// UpdateQueryStatus
No longer true, due to MarkAsRetrying


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/client-request-state.h@561
PS27, Line 561: initialized in InitExecRequest
No longer true. Also maybe note that the QueryDriver owns this.


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-beeswax-server.cc
File be/src/service/impala-beeswax-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-beeswax-server.cc@420
PS27, Line 420: Don't use
              :   // GetActiveQueryHandle because clients can request profiles for unregistered queries.
Not sure what this is supposed to mean


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-server.h
File be/src/service/impala-server.h:

http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-server.h@670
PS27, Line 670: query_driver
query_handle


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-server.h@868
PS27, Line 868:     /// query_state = union(beeswax::QueryState, ClientRequestState::RetryState).
Probably requires more explanation, eg. "used for the http server..."


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-server.h@985
PS27, Line 985: 'request_state'
needs to be updated


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-server.h@1024
PS27, Line 1024: ClientRequestState
needs to be updated


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-server.cc
File be/src/service/impala-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-server.cc@1160
PS24, Line 1160:   // unregistration work. StartUnregister() succeeds for the first thread to call it to
> This was based on https://gerrit.cloudera.org/#/c/15821/10 - basically ther
Ah okay, I hadn't noticed the equivalent started_finalize thing in ClientRequestState that this is replacing.

It might be nice to define like a QueryDriver::Finalize that calls StartUnregister and ClientRequestState::Finalize to encapsulate this in QueryDriver better, but not a big deal.


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-server.cc
File be/src/service/impala-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-server.cc@943
PS27, Line 943:   (*query_handle).EmplaceQueryDriver(this);
Similar to noted below, you could clean this up by adding a static QueryDriver::CreateNewDriver(ImpalaServer&, QueryHandle*)


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-server.cc@1177
PS27, Line 1177: &*
I think you could just pass a const QueryHandle& here and it would be cleaner


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-server.cc@1297
PS27, Line 1297: shared_ptr<QueryDriver>
So I believe this copies the shared_ptr, which results in an extra atomic inc/dec. It might be cleaner to just call GetQueryDriver() without calling SetQueryDriver() yet, do the nullptr check, and then call SetQueryDriver() below.

I also think it would be nice to have something like a static QueryDriver::CreateHandle(), which would maybe take like an ImpalaServer& for calling GetQueryDriver() and an out QueryHandle* and then if QueryDriver was a friend class of QueryHandle you could get rid of SetQueryDriver/EmplaceQueryDriver/SetClientRequestState


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-server.cc@1504
PS27, Line 1504: QueryHandle handle;
I think this is unnecessary and adds another shared_ptr copy. You can just pass 'query_handle' into GetActiveQueryHandle


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-server.cc@2411
PS27, Line 2411: query_handle
request_state?


http://gerrit.cloudera.org:8080/#/c/14824/27/tests/custom_cluster/test_query_retries.py
File tests/custom_cluster/test_query_retries.py:

http://gerrit.cloudera.org:8080/#/c/14824/27/tests/custom_cluster/test_query_retries.py@92
PS27, Line 92:   def test_kill_impalad_expect_retry(self):
I don't think I see a test case in here for if the retry is attempted due to an Exec() rpc failing


http://gerrit.cloudera.org:8080/#/c/14824/27/tests/custom_cluster/test_query_retries.py@363
PS27, Line 363:   def test_retry_query_hs2(self):
Fwiw, since beeswax is now deprecated and pretty much all clients use hs2, it would be preferable to have as more testing with hs2 vs. beeswax

A lot of these tests work out basically the same for both, so its not necessary to rewrite them all.



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 27
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Tue, 12 May 2020 19:32:09 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 24:

(8 comments)

http://gerrit.cloudera.org:8080/#/c/14824/26/be/src/runtime/coordinator.cc
File be/src/runtime/coordinator.cc:

http://gerrit.cloudera.org:8080/#/c/14824/26/be/src/runtime/coordinator.cc@511
PS26, Line 511: if (!failed_backend_states.empty()) {
> Seems like it shouldn't be possible for this to ever be false. Maybe add a 
Done


http://gerrit.cloudera.org:8080/#/c/14824/26/be/src/runtime/coordinator.cc@512
PS26, Line 512: RETURN_IF_ERROR
> I don't think its possible for HandleFailedExecRpcs() to return an error, a
Done


http://gerrit.cloudera.org:8080/#/c/14824/26/be/src/runtime/coordinator.cc@982
PS26, Line 982:     if (!retryable_status.ok()) {
> Probably fine to leave as is for now since currently any query that hits th
Responded in other comment


http://gerrit.cloudera.org:8080/#/c/14824/26/be/src/runtime/coordinator.cc@1079
PS26, Line 1079: 
> Normally, those two statuses would be the same, because ClientRequestState would get Coordinator's state returned to it from Exec(), which it would then set on itself, it won't set it anymore as we only preserve the first error message. I think that's messy and has the potential to be confusing for developers.

Not sure I follow.

After this patch and if a retry is triggered, the stack is: ClientRequestState::FinishExecQueryOrDmlRequest() --> Coordinator::Exec() --> Coordinator::StartBackendExec() --> Coordinator::HandleFailedExecRpcs() --> QueryDriver::TryQueryRetry() --> ClientRequestState::MarkAsRetrying() which sets ClientRequestState::query_status_.

Before this patch, if a query fails: ClientRequestState::FinishExecQueryOrDmlRequest() --> Coordinator::Exec() --> Coordinator::StartBackendExec() --> ClientRequestState::UpdateQueryStatus() which sets ClientRequestState::query_status_.

Yes, the stack is more complicated. I added some documentation to clarify it.

> it won't set it anymore as we only preserve the first error message

Right, but that seems to be a property of UpdateQueryStatus()'s existing behavior, and its called all over the place in ClientRequestState.

I agree the removing the circular dependency would be great to do, but I think it's debatable how much effort it is. At one point I had the call to TryQueryRetry inside UpdateQueryStatus, but there were a few bugs introduced when I did that. I can't remember exactly what they were, but needless to say this part of the code is sufficiently complex that even seemingly small changes can have unintended consequences.

Moreover, I personally want to defer the re-factoring changes to IMPALA-9370. I'm a bit wary of investing large chunks of time re-factoring this code multiple times, given the number of follow up tasks under IMPALA-9124, I think it is hard to predict how this code will evolve over time. A lot of the re-factoring we end up doing now might just be wasted work if we end up changing half the logic. Just a thought.


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h
File be/src/runtime/query-driver.h:

http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h@136
PS24, Line 136:   Status TryQueryRetry(ClientRequestState* client_request_state, Status* status,
> 'status' is used in one code path that calls TryQueryRetry

Not sure I follow. TryQueryRetry passes the status to MarkAsRetrying which sets the given status as the query_status_ of the ClientRequestState, which is what gets exposed in the runtime profile.

This was actually getting verified in the tests (up until recently) because they asserted that "Retryable error" was always in the "Query Status" field.

> but that's already the case since you're only adding the detail if we hit the max retries and not for other cases such as if rows had already been fetched, though maybe you want to do an AddDetail for those cases too.

I added it for the case when rows have already been fetched.


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h@204
PS24, Line 204:   std::unique_ptr<TExecRequest> retry_exec_request_;
> Personally, I think the code is way more confusing this way.
Deleted it.


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-hs2-server.cc
File be/src/service/impala-hs2-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-hs2-server.cc@884
PS24, Line 884:   QueryHandle query_handle;
> I understand that the point of QueryHandle is to make it harder for develop
I added overloads for "->" and "*". I tried my best to mirror the semantics of unique_ptr. It required quite a bit of re-factoring, and at some points some convoluted pointer manipulation, but I think it is correct.


http://gerrit.cloudera.org:8080/#/c/14824/26/common/thrift/generate_error_codes.py
File common/thrift/generate_error_codes.py:

http://gerrit.cloudera.org:8080/#/c/14824/26/common/thrift/generate_error_codes.py@467
PS26, Line 467:   ("RETRYABLE_ERROR", 151, "Retryable error: $0"),
> I'm not certain this is the right thing to do.
Removed. It was mainly an attempt to share a common error message prefix between the retryable errors, to make the messages more consistent, but I'm not sure it matters much.



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 24
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Tue, 12 May 2020 17:02:55 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 29:

(5 comments)

http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/runtime/query-driver.h
File be/src/runtime/query-driver.h:

http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/runtime/query-driver.h@58
PS27, Line 58:   return *request_state_;
> Sure, I mean there are basically two cases for how this might be used:
I think I see what your saying now. Yeah, your suggestion makes sense since the caller can always decide if they want to make a copy.


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h
File be/src/runtime/query-driver.h:

http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h@136
PS24, Line 136: /// single query submission.
> > I think in that case the error still
ahh I see what you are saying now - right, the code path in UpdateBackendExecStatus where AuxErrorInfoPB is present, but no error is returned from the code path. the 'retryable_status' gets dropped after the call to TryQueryRetry. looks like this happens in CancelFromThreadPool and HandleFailedExecRpcs as well.
filed IMPALA-9748 to follow up on this


http://gerrit.cloudera.org:8080/#/c/14824/28/be/src/runtime/query-driver.cc
File be/src/runtime/query-driver.cc:

http://gerrit.cloudera.org:8080/#/c/14824/28/be/src/runtime/query-driver.cc@35
PS28, Line 35: Finalize() must
> Finalize()
Done


http://gerrit.cloudera.org:8080/#/c/14824/28/be/src/service/impala-hs2-server.cc
File be/src/service/impala-hs2-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/28/be/src/service/impala-hs2-server.cc@150
PS28, Line 150:   if (!register_status.ok()) {
> You could also move the call to CreateClientRequestState into CreateNewDriv
Done


http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-server.cc
File be/src/service/impala-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/27/be/src/service/impala-server.cc@1177
PS27, Line 1177: in
> In that case, a non-const QueryHandle& or a QueryHandle*. Not a big deal th
Actually, your original suggestion does work. The fact that QueryHandle is a const doesn't really matter because "->" returns a non-const pointer to ClientRequestState. So I re-factored CloseClientRequestState, UpdateExecSummary, and ArchiveQuery to all take in a const QueryHandle&.



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 29
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Thu, 14 May 2020 20:49:45 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 29:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/6067/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 29
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Thu, 14 May 2020 21:41:06 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 19:

Rebased on top of IMPALA-9380, which had a bunch of merge conflicts that I had to resolve. Re-ran the tests from IMPALA-9380 and they pass.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 19
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Wed, 06 May 2020 03:13:06 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 19:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/14824/19/be/src/service/impala-server.cc
File be/src/service/impala-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/19/be/src/service/impala-server.cc@1153
PS19, Line 1153:   
line has trailing whitespace


http://gerrit.cloudera.org:8080/#/c/14824/19/be/src/service/impala-server.cc@1310
PS19, Line 1310:   RETURN_IF_ERROR(query_handle.request_state->Cancel(/*check_inflight=*/ true, /*cause=*/ nullptr));
line too long (100 > 90)



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 19
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Wed, 06 May 2020 03:10:49 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 8:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/5663/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 8
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Tue, 31 Mar 2020 18:39:40 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 18:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/5971/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 18
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Wed, 06 May 2020 02:10:43 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has removed Michael Ho from this change.  ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Removed reviewer Michael Ho.
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: deleteReviewer
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 2
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 31:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/6077/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 31
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 15 May 2020 04:01:38 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 22:

Build Failed 

https://jenkins.impala.io/job/gerrit-code-review-checks/5982/ : Initial code review checks failed. See linked job for details on the failure.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 22
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Thu, 07 May 2020 00:41:07 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 32:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/6082/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 32
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 15 May 2020 15:26:22 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 32:

(1 comment)

Fixing a few issues that are causing the test failures:

(1) Removed a DCHECK (see comment below)
(2) Fixed a bug where GetQueryHandle was not passing return_unregistered to GetQueryDriver

Ran a bunch of more tests locally, and things look good:
* Re-ran the stress tests for tpcds and tpch
* Looped the new tests overnight and no failures

Did some additional cleanup of code comments, removing unnecessary imports, minor-refactoring, etc.

http://gerrit.cloudera.org:8080/#/c/14824/26/be/src/runtime/coordinator.cc
File be/src/runtime/coordinator.cc:

http://gerrit.cloudera.org:8080/#/c/14824/26/be/src/runtime/coordinator.cc@511
PS26, Line 511: if (!failed_backend_states.empty()) {
> Done
I had to remove the DCHECK because it caused a crash. Turns out that because of the IsAborted() check above, its possible for this list to be empty. Tests that use the EXEC_SERIALIZE_FRAGMENT_INFO Debug Action trigger the DCHECK.



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 32
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 15 May 2020 14:47:57 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#6).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes.

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the leaving node, queries are retried), and (2) if a
query fails and as a result, blacklists a node. Both events are
considered cluster membership changes as they affect what nodes a query
will be scheduled on.

Implementation:
* Query retries are driven by a dedicated threadpool
    * ImpalaServer::RetryQueryFromThreadPool implements the core logic to
      actually retry a failed query.
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A query cannot be retried once any results from the original query
  have been fetched, this is to prevent users from seeing incorrect results

Features:
* Retries are transparent to the user
    * This is achieved by adding a mapping from failed query ids to the
      query id of the retried query
    * ImpalaServer uses this mapping in GetClientFacingRequestState
      which is used to differentiate between "client facing" requests
      for a ClientRequestState vs. internal requrets for a CRS
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
    * "Impala Query Status" is a new field in runtime profiles that
      displays the ClientRequestState execution state (which includes
      the RETRYING and RETRIED states)
    * The Impala Web UI will list all retried queries as being in the
      "RETRIED" state
* Retried queries skip all fe/ planning, authorization, etc.
* This feature is configurable ('retry_failed_queries') and is off by
  default

Refactoring:
* Changes the ClientRequestState so that it can take in an existing
  TExecRequest
    * This is required when retrying queries because the
      TExecRequest of the failed query is copied and used for the
      ClientRequestState of the retried query
* ClientRequestState::ExecState is extended with three new states:
  RETRYING, RETRIED, and UNKNOWN.

Testing:
* Added integration tests in test_query_retries.py, these tests
  consistently pass when run locally
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures

TODO:
* Additional re-factoring / code cleanup
* Lots more documentation

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
A be/src/service/retry-work.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M common/thrift/generate_error_codes.py
A tests/custom_cluster/test_query_retries.py
16 files changed, 793 insertions(+), 168 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/6
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 6
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Joe McDonnell, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#25).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes (IMPALA-9124).

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the removed node, queries are retried), or (2) if a
query fails and as a result, blacklists a node. Either event is
considered a cluster membership change as it affects what nodes a query
will be scheduled on. The assumption is that a retry of the query with
the updated cluster membership will succeed.

A query retry is modelled as a brand new query, with its own query id.
This simplifies the implementation and the resulting runtime profiles
when queries are retried.

Core Features:
* Retries are transparent to the user; no modification to client
  libraries are necessary to support query retries
* Retried queries skip all fe/ parsing, planning, authorization, etc.
* Retries are configurable ('retry_failed_queries') and are off by
  default

Implementation:
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A new layer of abstraction between the ImpalaServer and
  ClientRequestState has been added; it is called the QueryDriver
* Each ClientRequestState is treated as a single attempt of a query, and
  the QueryDriver owns all ClientRequestStates for a query
* ClientRequestState has a new state object called RetryState; a
  ClientRequestState can either be NOT_RETRIED, RETRYING, or RETRIED
* The QueryDriver owns the TExecRequest for the query as well, it is
  re-used for each query retry

Observability:
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
* Runtime profiles of queries that fail and then are retried will have:
    * "Retry Status: RETRIED"
    * "Retry Cause: [the error that triggered the retry]"
    * "Retried Query Id: [the query id of the retried query]"
* Runtime profiles of the retried query (e.g. the second attempt of the
  query) will include:
    * "Original Query Id: [the query id of the original query]"
* The Impala Web UI will list all retried queries as being in the
  "RETRIED" state

Testing:
* Added E2E tests in test_query_retries.py; looped tests for a few days
* Added a stress test query_retries_stress_runner.py that runs concurrent
  streams of a TPC-{H,DS} workload and randomly kills impalads
* Ran the stress test with various configurations: tpch on parquet,
  tpcds on parquet, tpch 30 GB on parquet (one stream), tpcds 30 GB on
  parquet (one stream), tpch on text, tpcds on text
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures
* Ran 30 GB TPC-DS workload on a 3 node cluster, randomly restarted
  impalads, and manually verified that queries were retried
* Manually tested retries work with various clients, specifically the
  impala-shell and Hue
* Ran core tests and query retry stress test against an ASAN build
* Ran concurrent_select.py to stress query cancellation
* Ran be/ tests against a TSAN build, filed IMPALA-9730 as a follow up

Limitations:
* There are several limitations that are listed out in the parent JIRA

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/benchmarks/process-wide-locks-benchmark.cc
M be/src/runtime/CMakeLists.txt
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
A be/src/runtime/query-driver.cc
A be/src/runtime/query-driver.h
M be/src/service/CMakeLists.txt
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/control-service.cc
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
R be/src/service/query-driver-map.cc
A be/src/service/query-driver-map.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/testutil/impalad-query-executor.cc
M be/src/testutil/impalad-query-executor.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M common/thrift/generate_error_codes.py
M tests/common/impala_cluster.py
M tests/common/impala_service.py
A tests/custom_cluster/test_query_retries.py
A tests/stress/query_retries_stress_runner.py
28 files changed, 2,480 insertions(+), 513 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/25
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 25
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#9).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes (IMPALA-9124).

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the removed node, queries are retried), or (2) if a
query fails and as a result, blacklists a node. Either event is
considered a cluster membership change as it affects what nodes a query
will be scheduled on. The assumption is that a retry of the query with
the updated cluster membership will succeed.

A query retry is modelled as a brand new query, with its own query id.
This simplifies the implementation and the resulting runtime profiles
when queries are retried.

Core Features:
* Retries are transparent to the user; no modification to client
  libraries are necessary to support query retries
* Retried queries skip all fe/ parsing, planning, authorization, etc.
* Retries are configurable ('retry_failed_queries') and are off by
  default

Implementation:
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A new layer of abstraction between the ImpalaServer and
  ClientRequestState has been added; it is called the QueryDriver
* Each ClientRequestState is treated as a single attempt of a query, and
  the QueryDriver owns all ClientRequestStates for a query
* ClientRequestState has a new state object called RetryState; a
  ClientRequestState can either be NOT_RETRIED, RETRYING, or RETRIED
* The QueryDriver owns the TExecRequest for the query as well, it is
  re-used for each query retry

Observability:
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
* Runtime profiles of queries that fail and then are retried will have:
    * "Retry Status: RETRIED"
    * "Retry Cause: [the error that triggered the retry]"
    * "Retried Query Id: [the query id of the retried query]"
* Runtime profiles of the retried query (e.g. the second attempt of the
  query) will include:
    * "Original Query Id: [the query id of the original query]"
* The Impala Web UI will list all retried queries as being in the
  "RETRIED" state

Testing:
* Added E2E tests in test_query_retries.py
* Added a stress test query_retries_stress_runner.py that runs concurrent
  streams of a TPC workload and randomly kills impalads
* Ran the stress test with various configurations: tpch on parquet,
  tpcds on parquet, tpch 30 GB on parquet (one stream), tpcds 30 GB on
  parquet (one stream), tpch on text, tpcds on text
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures
* Ran 30 GB TPC-DS workload on a 3 node cluster, randomly restarted
  impalads, and manually verified that queries were retried
* Manually tested retries work with various clients, specifically the
  impala-shell and Hue

Limitations:
* IMPALA-9225: A query cannot be retried once any results from the original
  query have been fetched
* IMPALA-9200: A query can currently only be retried once

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/runtime/CMakeLists.txt
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
A be/src/runtime/query-driver.cc
A be/src/runtime/query-driver.h
M be/src/service/CMakeLists.txt
D be/src/service/client-request-state-map.h
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/control-service.cc
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
R be/src/service/query-driver-map.cc
A be/src/service/query-driver-map.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M common/thrift/generate_error_codes.py
M tests/common/impala_cluster.py
M tests/common/impala_service.py
A tests/custom_cluster/test_query_retries.py
A tests/stress/query_retries_stress_runner.py
26 files changed, 2,253 insertions(+), 448 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/9
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 9
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 6:

Build Failed 

https://jenkins.impala.io/job/gerrit-code-review-checks/5184/ : Initial code review checks failed. See linked job for details on the failure.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 6
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Mon, 10 Feb 2020 17:34:03 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Joe McDonnell, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#28).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes (IMPALA-9124).

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the removed node, queries are retried), or (2) if a
query fails and as a result, blacklists a node. Either event is
considered a cluster membership change as it affects what nodes a query
will be scheduled on. The assumption is that a retry of the query with
the updated cluster membership will succeed.

A query retry is modelled as a brand new query, with its own query id.
This simplifies the implementation and the resulting runtime profiles
when queries are retried.

Core Features:
* Retries are transparent to the user; no modification to client
  libraries are necessary to support query retries
* Retried queries skip all fe/ parsing, planning, authorization, etc.
* Retries are configurable ('retry_failed_queries') and are off by
  default

Implementation:
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A new layer of abstraction between the ImpalaServer and
  ClientRequestState has been added; it is called the QueryDriver
* Each ClientRequestState is treated as a single attempt of a query, and
  the QueryDriver owns all ClientRequestStates for a query
* ClientRequestState has a new state object called RetryState; a
  ClientRequestState can either be NOT_RETRIED, RETRYING, or RETRIED
* The QueryDriver owns the TExecRequest for the query as well, it is
  re-used for each query retry

Observability:
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
* Runtime profiles of queries that fail and then are retried will have:
    * "Retry Status: RETRIED"
    * "Retry Cause: [the error that triggered the retry]"
    * "Retried Query Id: [the query id of the retried query]"
* Runtime profiles of the retried query (e.g. the second attempt of the
  query) will include:
    * "Original Query Id: [the query id of the original query]"
* The Impala Web UI will list all retried queries as being in the
  "RETRIED" state

Testing:
* Added E2E tests in test_query_retries.py; looped tests for a few days
* Added a stress test query_retries_stress_runner.py that runs concurrent
  streams of a TPC-{H,DS} workload and randomly kills impalads
* Ran the stress test with various configurations: tpch on parquet,
  tpcds on parquet, tpch 30 GB on parquet (one stream), tpcds 30 GB on
  parquet (one stream), tpch on text, tpcds on text
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures
* Ran 30 GB TPC-DS workload on a 3 node cluster, randomly restarted
  impalads, and manually verified that queries were retried
* Manually tested retries work with various clients, specifically the
  impala-shell and Hue
* Ran core tests and query retry stress test against an ASAN build
* Ran concurrent_select.py to stress query cancellation
* Ran be/ tests against a TSAN build, filed IMPALA-9730 as a follow up

Limitations:
* There are several limitations that are listed out in the parent JIRA

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/benchmarks/process-wide-locks-benchmark.cc
M be/src/runtime/CMakeLists.txt
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
A be/src/runtime/query-driver.cc
A be/src/runtime/query-driver.h
M be/src/service/CMakeLists.txt
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/control-service.cc
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
R be/src/service/query-driver-map.cc
A be/src/service/query-driver-map.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/testutil/impalad-query-executor.cc
M be/src/testutil/impalad-query-executor.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M tests/common/impala_cluster.py
M tests/common/impala_service.py
A tests/custom_cluster/test_query_retries.py
A tests/stress/query_retries_stress_runner.py
27 files changed, 2,692 insertions(+), 682 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/28
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 28
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 2:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/14824/2/be/src/service/impala-server.h
File be/src/service/impala-server.h:

http://gerrit.cloudera.org:8080/#/c/14824/2/be/src/service/impala-server.h@1007
PS2, Line 1007:   /// order to avoid query compilation and planning again. Once the new query is registered
line too long (91 > 90)



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 2
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 24 Jan 2020 16:50:48 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 28:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/6045/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 28
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Tue, 12 May 2020 23:53:22 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 12:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/5916/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 12
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Thu, 30 Apr 2020 00:50:28 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Joe McDonnell, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#32).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes (IMPALA-9124).

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the removed node, queries are retried), or (2) if a
query fails and as a result, blacklists a node. Either event is
considered a cluster membership change as it affects what nodes a query
will be scheduled on. The assumption is that a retry of the query with
the updated cluster membership will succeed.

A query retry is modelled as a brand new query, with its own query id.
This simplifies the implementation and the resulting runtime profiles
when queries are retried.

Core Features:
* Retries are transparent to the user; no modification to client
  libraries are necessary to support query retries
* Retried queries skip all fe/ parsing, planning, authorization, etc.
* Retries are configurable ('retry_failed_queries') and are off by
  default

Implementation:
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A new layer of abstraction between the ImpalaServer and
  ClientRequestState has been added; it is called the QueryDriver
* Each ClientRequestState is treated as a single attempt of a query, and
  the QueryDriver owns all ClientRequestStates for a query
* ClientRequestState has a new state object called RetryState; a
  ClientRequestState can either be NOT_RETRIED, RETRYING, or RETRIED
* The QueryDriver owns the TExecRequest for the query as well, it is
  re-used for each query retry
* QueryDrivers and ClientRequestStates are now referenced using a
  QueryHandle

Observability:
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
* Runtime profiles of queries that fail and then are retried will have:
    * "Retry Status: RETRIED"
    * "Retry Cause: [the error that triggered the retry]"
    * "Retried Query Id: [the query id of the retried query]"
* Runtime profiles of the retried query (e.g. the second attempt of the
  query) will include:
    * "Original Query Id: [the query id of the original query]"
* The Impala Web UI will list all retried queries as being in the
  "RETRIED" state

Testing:
* Added E2E tests in test_query_retries.py; looped tests for a few days
* Added a stress test query_retries_stress_runner.py that runs concurrent
  streams of a TPC-{H,DS} workload and randomly kills impalads
* Ran the stress test with various configurations: tpch on parquet,
  tpcds on parquet, tpch 30 GB on parquet (one stream), tpcds 30 GB on
  parquet (one stream), tpch on text, tpcds on text
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures
* Ran 30 GB TPC-DS workload on a 3 node cluster, randomly restarted
  impalads, and manually verified that queries were retried
* Manually tested retries work with various clients, specifically the
  impala-shell and Hue
* Ran core tests and query retry stress test against an ASAN build
* Ran concurrent_select.py to stress query cancellation
* Ran be/ tests against a TSAN build

Limitations:
* There are several limitations that are listed out in the parent JIRA

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/benchmarks/process-wide-locks-benchmark.cc
M be/src/runtime/CMakeLists.txt
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
A be/src/runtime/query-driver.cc
A be/src/runtime/query-driver.h
M be/src/service/CMakeLists.txt
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/control-service.cc
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
R be/src/service/query-driver-map.cc
A be/src/service/query-driver-map.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/testutil/impalad-query-executor.cc
M be/src/testutil/impalad-query-executor.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M tests/common/impala_cluster.py
M tests/common/impala_service.py
A tests/custom_cluster/test_query_retries.py
A tests/stress/query_retries_stress_runner.py
27 files changed, 2,700 insertions(+), 696 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/32
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 32
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 11:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/5915/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 11
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Thu, 30 Apr 2020 00:48:34 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 6:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/14824/6/be/src/service/impala-server.cc
File be/src/service/impala-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/6/be/src/service/impala-server.cc@1463
PS6, Line 1463: void ImpalaServer::BlockOnWait(shared_ptr<ClientRequestState>* request_state, bool* timed_out,
> line too long (94 > 90)
Done



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 6
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Mon, 10 Feb 2020 16:51:50 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 9:

(2 comments)

responded to a few comments from Thomas, still thinking through some of his other comments.

http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/runtime/coordinator.cc
File be/src/runtime/coordinator.cc:

http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/runtime/coordinator.cc@922
PS9, Line 922:           parent_query_driver_->TryQueryRetry(parent_request_state_, &retryable_status));
yeah, agree it is unfortunate. although i think the issue has always been there. the coordinator class already has a parent_request_state_ that points to the owning ClientRequestState. the ClientRequestState has a similar pattern, it has a parent_server_ which points to the owning ImpalaServer.
i'm not sure this is overall a major problem, it does make the code a bit confusing, but I've done my best to limit the usage of the parent_query_driver_ class. only the TryQueryRetry method is ever called on the parent_query_driver_.
yeah, it does take the coordinator lock, but only for a pretty short period of time - it just checks some state. the actual work to schedule and run the retried query is done in a separate thread, that doesn't require holding the coordinator lock. acquiring the lock should only be done rarely - only when a query fails with a retryable error.

> Would it be possible to instead have the QueryDriver wait on the coordinator to finish and then check its status and decide whether to retry then?

> One problem is the QueryDriver needs to know not just if the query hit an error but if the error was something retryable, but we could do something like have the coordinator remember any nodes it blacklists and expose that info to the QueryDriver.

yeah, I considered this at some point, and it might be a cleaner design, just not sure how much re-factoring it is going to require.
i can convince myself the current approach of just calling TryRetryQuery in the coordinator is fine as well. I think they can be thought of as "callback" functions into the QueryDriver that get triggered under specific conditions.

if you still feel strongly about it, i can do some digging, but would prefer to make the code changes in a follow up patch because i don't think it will be a straightforward / small change to make, and i don't want to expand the scope of this patch further.


http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/runtime/coordinator.cc@1060
PS9, Line 1060:   ExecEnv::GetInstance()->cluster_membership_mgr()->BlacklistExecutor(
> This of course doesn't actually guarantee that the retried query won't be s
that's a good point, filed IMPALA-9636 to fix this.



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 9
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Thu, 09 Apr 2020 17:57:32 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Thomas Tauber-Marshall (Code Review)" <ge...@cloudera.org>.
Thomas Tauber-Marshall has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 7:

(1 comment)

> Instead of creating a separate doc, think it might be easier to
 > just post comments here.
 > 
 > I've done some re-factoring locally and this is what I have so far:
 > 
 > * Created a "QueryDriver", which (as suggested by Thomas) is a
 > layer of abstraction of on top of ClientRequestState
 > * "QueryDriver" owns the ClientRequestState; all access to the
 > ClientRequestState goes through the QueryDriver
 > * Since the first patch will only support a single retry, the
 > QueryDriver has a regular client_request_state_ and a
 > retried_client_request_state_ (one for the original query and the
 > other for the retried query)
 > * QueryDriver owns the TExecRequest that is used by both the
 > original and retried queries
 > * The "transparent" part of this feature is now handled by the
 > QueryDriver (since the QueryDriver owns all access to the
 > ClientRequestState)
 > * The QueryDriver has two methods: GetActiveClientRequestState()
 > and GetClientRequestState(query_id)
 > * GetActiveClientRequestState() returns the CRS for the "active"
 > query - e.g. if the query has not been retried yet, it returns the
 > original query; if the query has been retried, it returns the
 > retried query
 > * GetClientRequestState(query_id) returns the CRS for the query
 > that matches the given 'query_id' - this allows clients to get the
 > CRS of exactly the query_id they are looking for
 > * ImpalaServer creates a QueryDriver for each query; the
 > ImpalaServer::client_request_state_map_ has been replaced with the
 > ImpalaServer::query_driver_map_ which maps query ids to
 > QueryDrivers
 > * The query_driver_map_ can have key, values pairs with the same
 > value (e.g. the same QueryDriver)
 > * Moved all of the retry code (along with the retry threadpool)
 > into QueryDriver (specifically QueryDriver::RetryQueryFromThreadPool)
 > 
 > Example:
 > 
 > * Client submits a query for execution
 > * Impala creates a QueryDriver for the query
 > * QueryDriver::CreateClientRequestState creates a ClientRequestState
 > for the query (this CRS is owned by the QueryDriver)
 > * ImpalaServer::RegisterQuery "registers" the query and adds an
 > entry to ImpalaServer::query_driver_map_ that maps the query id to
 > the QueryDriver
 > * QueryDriver::RunFrontendPlanner creates the TExecRequest (owned
 > by the QueryDriver)
 > * Eventually, the query fails and is retried
 > * QueryDriver::RetryQueryFromThreadPool contains all the retry
 > logic (moved from ImpalaServer)
 > * It cancels the original query, creates the new one and runs it
 > * It calls ImpalaServer::RegisterQuery again, but with the retried
 > query id
 > * It manipulates some internal pointers so that retried_client_request_state_
 > now points to the CRS of the retried query

That all sounds good to me

http://gerrit.cloudera.org:8080/#/c/14824/5/be/src/service/impala-server.cc
File be/src/service/impala-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/5/be/src/service/impala-server.cc@1025
PS5, Line 1025: query_ctx->query_id = UuidToQueryId(random_generator()());
> I'll try thinking through this some more. One challenge with creating an in
Sure, I think this can probably be deferred for now and we can go with your original approach.



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 7
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Mon, 24 Feb 2020 21:25:44 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 22: Code-Review+1

(2 comments)

Carrying +1. Addressed Joe's comments. Ran TSAN tests and fixed a lock ordering issue reported by TSAN.

http://gerrit.cloudera.org:8080/#/c/14824/21/be/src/runtime/coordinator.cc
File be/src/runtime/coordinator.cc:

http://gerrit.cloudera.org:8080/#/c/14824/21/be/src/runtime/coordinator.cc@1071
PS21, Line 1071:   for (BackendState* backend_state : failed_backend_states) {
               :     backend_addresses.push_back(
               :         TNetworkAddressToString(backend_state->krpc_impalad_address()));
               :   }
> Purely style nit: I think this would be cleaner as a for-loop.
Done


http://gerrit.cloudera.org:8080/#/c/14824/21/be/src/runtime/coordinator.cc@1077
PS21, Line 1077:   for (BackendState* backend_state : failed_backend_states) {
               :     retryable_status.MergeStatus(
               :         FromKuduStatus(backend_state->exec_rpc_status()), "Exec() rpc failed");
               :   }
               : 
> Purely style nit: I think this would be cleaner as a for-loop:
Done



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 22
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Thu, 07 May 2020 00:19:41 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has uploaded a new patch set (#2). ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes.

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the leaving node, queries are retried), and (2) if a
query fails and as a result, blacklists a node. Both events are
considered cluster membership changes as they affect what nodes a query
will be scheduled on.

Implementation:
* Query retries are driven by a dedicated threadpool
    * ImpalaServer::RetryQueryFromThreadPool implements the core logic to
      actually retry a failed query.
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A query cannot be retried once any results from the original query
  have been fetched, this is to prevent users from seeing incorrect results

Features:
* Retries are transparent to the user
    * This is achieved by adding a mapping from failed query ids to the
      query id of the retried query
    * ImpalaServer uses this mapping in GetClientFacingRequestState
      which is used to differentiate between "client facing" requests
      for a ClientRequestState vs. internal requrets for a CRS
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
    * "Impala Query Status" is a new field in runtime profiles that
      displays the ClientRequestState execution state (which includes
      the RETRYING and RETRIED states)
    * The Impala Web UI will list all retried queries as being in the
      "RETRIED" state
* Retried queries skip all fe/ planning, authorization, etc.
* This feature is configurable ('retry_failed_queries') and is off by
  default

Refactoring:
* Changes the ClientRequestState so that it can take in an existing
  TExecRequest
    * This is required when retrying queries because the
      TExecRequest of the failed query is copied and used for the
      ClientRequestState of the retried query
* ClientRequestState::ExecState is extended with three new states:
  RETRYING, RETRIED, and UNKNOWN.

Testing:
* Added integration tests in test_query_retries.py, these tests
  consistently pass when run locally
* Ran exhaustive tests

TODO:
* There are some failed tests I am working through
* Additional re-factoring / code cleanup
* Lots more documentation

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
M be/src/service/client-request-state-map.h
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
A be/src/service/retry-work.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
A tests/custom_cluster/test_query_retries.py
16 files changed, 808 insertions(+), 167 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/2
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 2
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Michael Ho <mi...@gmail.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Joe McDonnell, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#22).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes (IMPALA-9124).

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the removed node, queries are retried), or (2) if a
query fails and as a result, blacklists a node. Either event is
considered a cluster membership change as it affects what nodes a query
will be scheduled on. The assumption is that a retry of the query with
the updated cluster membership will succeed.

A query retry is modelled as a brand new query, with its own query id.
This simplifies the implementation and the resulting runtime profiles
when queries are retried.

Core Features:
* Retries are transparent to the user; no modification to client
  libraries are necessary to support query retries
* Retried queries skip all fe/ parsing, planning, authorization, etc.
* Retries are configurable ('retry_failed_queries') and are off by
  default

Implementation:
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A new layer of abstraction between the ImpalaServer and
  ClientRequestState has been added; it is called the QueryDriver
* Each ClientRequestState is treated as a single attempt of a query, and
  the QueryDriver owns all ClientRequestStates for a query
* ClientRequestState has a new state object called RetryState; a
  ClientRequestState can either be NOT_RETRIED, RETRYING, or RETRIED
* The QueryDriver owns the TExecRequest for the query as well, it is
  re-used for each query retry

Observability:
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
* Runtime profiles of queries that fail and then are retried will have:
    * "Retry Status: RETRIED"
    * "Retry Cause: [the error that triggered the retry]"
    * "Retried Query Id: [the query id of the retried query]"
* Runtime profiles of the retried query (e.g. the second attempt of the
  query) will include:
    * "Original Query Id: [the query id of the original query]"
* The Impala Web UI will list all retried queries as being in the
  "RETRIED" state

Testing:
* Added E2E tests in test_query_retries.py; looped tests for a few days
* Added a stress test query_retries_stress_runner.py that runs concurrent
  streams of a TPC workload and randomly kills impalads
* Ran the stress test with various configurations: tpch on parquet,
  tpcds on parquet, tpch 30 GB on parquet (one stream), tpcds 30 GB on
  parquet (one stream), tpch on text, tpcds on text
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures
* Ran 30 GB TPC-DS workload on a 3 node cluster, randomly restarted
  impalads, and manually verified that queries were retried
* Manually tested retries work with various clients, specifically the
  impala-shell and Hue
* Ran core tests and query retry stress test against an ASAN build
* Ran concurrent_select.py to stress query cancellation
* Ran be/ tests against a TSAN build, filed IMPALA-9730 as a follow up

Limitations:
* There are several limitations that are listed out in the parent JIRA

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/benchmarks/process-wide-locks-benchmark.cc
M be/src/runtime/CMakeLists.txt
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
A be/src/runtime/query-driver.cc
A be/src/runtime/query-driver.h
M be/src/service/CMakeLists.txt
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/control-service.cc
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
R be/src/service/query-driver-map.cc
A be/src/service/query-driver-map.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/testutil/impalad-query-executor.cc
M be/src/testutil/impalad-query-executor.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M common/thrift/generate_error_codes.py
M tests/common/impala_cluster.py
M tests/common/impala_service.py
A tests/custom_cluster/test_query_retries.py
A tests/stress/query_retries_stress_runner.py
28 files changed, 2,458 insertions(+), 510 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/22
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 22
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 26:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/6012/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 26
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 08 May 2020 20:07:10 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#8).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes (IMPALA-9124).

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the removed node, queries are retried), or (2) if a
query fails and as a result, blacklists a node. Either event is
considered a cluster membership change as it affects what nodes a query
will be scheduled on. The assumption is that a retry of the query with
the updated cluster membership will succeed.

A query retry is modelled as a brand new query, with its own query id.
This simplifies the implementation and the resulting runtime profiles
when queries are retried.

Core Features:
* Retries are transparent to the user; no modification to client
  libraries are necessary to support query retries
* Retried queries skip all fe/ parsing, planning, authorization, etc.
* Retries are configurable ('retry_failed_queries') and are off by
  default

Implementation:
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A new layer of abstraction between the ImpalaServer and
  ClientRequestState has been added; it is called the QueryDriver
* Each ClientRequestState is treated as a single attempt of a query, and
  the QueryDriver owns all ClientRequestStates for a query
* ClientRequestState has a new state object called RetryState; a
  ClientRequestState can either be NOT_RETRIED, RETRYING, or RETRIED
* The QueryDriver owns the TExecRequest for the query as well, it is
  re-used for each query retry

Observability:
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
* Runtime profiles of queries that fail and then are retried will have:
    * "Retry Status: RETRIED"
    * "Retry Cause: [the error that triggered the retry]"
    * "Retried Query Id: [the query id of the retried query]"
* Runtime profiles of the retried query (e.g. the second attempt of the
  query) will include:
    * "Original Query Id: [the query id of the original query]"
* The Impala Web UI will list all retried queries as being in the
  "RETRIED" state

Testing:
* Added E2E tests in test_query_retries.py
* Added a stress test query_retries_stress_runner.py that runs concurrent
  streams of a TPC workload and randomly kills impalads
* Ran the stress test with various configurations: tpch on parquet,
  tpcds on parquet, tpch 30 GB on parquet (one stream), tpcds 30 GB on
  parquet (one stream), tpch on text, tpcds on text
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures
* Ran 30 GB TPC-DS workload on a 3 node cluster, randomly restarted
  impalads, and manually verified that queries were retried
* Manually tested retries work with various clients, specifically the
  impala-shell and Hue

Limitations:
* IMPALA-9225: A query cannot be retried once any results from the original
  query have been fetched
* IMPALA-9200: A query can currently only be retried once

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/runtime/CMakeLists.txt
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
A be/src/runtime/query-driver.cc
A be/src/runtime/query-driver.h
M be/src/service/CMakeLists.txt
D be/src/service/client-request-state-map.h
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/control-service.cc
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
R be/src/service/query-driver-map.cc
A be/src/service/query-driver-map.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M common/thrift/generate_error_codes.py
M tests/common/impala_cluster.py
M tests/common/impala_service.py
A tests/custom_cluster/test_query_retries.py
A tests/stress/query_retries_stress_runner.py
26 files changed, 2,253 insertions(+), 448 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/8
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 8
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 7:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/5186/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 7
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Mon, 10 Feb 2020 17:36:57 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 5:

(7 comments)

Addressed some of the comments and filed a few follow up JIRAs. Going to start a doc to discuss the "transparent" part of this feature (e.g. the internal query ids). Working on doing some of the re-factoring you mentioned (e.g. adding another layer of abstraction on top of ClientRequestState).

http://gerrit.cloudera.org:8080/#/c/14824/5//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/14824/5//COMMIT_MSG@26
PS5, Line 26: A query cannot be retried once any results from the original query
            :   have been fetched, this is to prevent users from seeing incorrect results
> Do we have any mechanism for a user to specify that they want their results
Not yet, but planning to tackle that in a follow up JIRA: IMPALA-9225


http://gerrit.cloudera.org:8080/#/c/14824/5//COMMIT_MSG@41
PS5, Line 41: The Impala Web UI will list all retried queries as being in the
            :       "RETRIED" state
> This may be a reasonable approach, but I think we should think about the us
Yeah, these are legitimate issues. We have few follow up JIRAs to tackle some of these problems: IMPALA-9229, IMPALA-9230, IMPALA-9213 and I just filed IMPALA-9364 as well - open to more suggestions about how to improve supportability.

For this patch, I think it should be sufficient to add the original query id to the retried query profile, and vice versa.


http://gerrit.cloudera.org:8080/#/c/14824/5/be/src/service/client-request-state-map.h
File be/src/service/client-request-state-map.h:

http://gerrit.cloudera.org:8080/#/c/14824/5/be/src/service/client-request-state-map.h@38
PS5, Line 38: bool overwrite = false
> I don't think this is being used anywhere anymore.
Done


http://gerrit.cloudera.org:8080/#/c/14824/5/be/src/service/client-request-state.h
File be/src/service/client-request-state.h:

http://gerrit.cloudera.org:8080/#/c/14824/5/be/src/service/client-request-state.h@82
PS5, Line 82: RETRYING, RETRIED, UNKNOWN
> I think this approach, and the way you have to save the previous state and 
Yeah, thats a good suggestion. Haven't looked into adding the ClientRequestState wrapper yet, but for now just split out the the RETRYING and RETRIED states into a `enum RetryState`. It actually simplifies the state management considerably, so thanks for the suggestion.


http://gerrit.cloudera.org:8080/#/c/14824/5/be/src/service/impala-server.cc
File be/src/service/impala-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/5/be/src/service/impala-server.cc@632
PS5, Line 632:     shared_ptr<ClientRequestState> request_state = GetClientFacingRequestState(query_id);
> Unless I'm missing something in this patch, this means that only the profil
hmm not sure why I did that, changed back to GetClientRequestState


http://gerrit.cloudera.org:8080/#/c/14824/5/be/src/service/impala-server.cc@920
PS5, Line 920:   unique_ptr<TExecRequest> exec_request = make_unique<TExecRequest>();
> So I'm wondering if this patch would benefit from adding another layer of a
I think that makes sense. IMO the ImpalaServer <--> ClientRequestState <--> Coordinator code needs some re-factoring in general.

I wanted to some general re-factoring all this code (ImpalaServer, ClientRequestState, Coordinator) so I filed IMPALA-9370 as a follow up, but I think it makes sense to move some of the retry logic into its own class in this patch.

I re-factored the BlockOnWait code so that it isn't duplicated across the hs2 and beeswax server.


http://gerrit.cloudera.org:8080/#/c/14824/5/be/src/service/impala-server.cc@1025
PS5, Line 1025: query_ctx->query_id = UuidToQueryId(random_generator()());
> As mentioned before, I definitely think it would be awesome if we could cre
I sort of did this in the previous patch update. I added a client_query_id_mapping_ that allows an optional mapping between query ids. It doesn't exactly create an "internal" id, but its cleaner that what I was doing before.

I think I'm going to resurrect the design doc for this feature and write up some notes on how best to approach this. We can finalize the design there.

The mixing of the hi/lo of the TUniqueId is an interesting idea. I'll be sure to include that.



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 5
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Mon, 10 Feb 2020 16:50:40 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Thomas Tauber-Marshall (Code Review)" <ge...@cloudera.org>.
Thomas Tauber-Marshall has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 26:

(8 comments)

http://gerrit.cloudera.org:8080/#/c/14824/26/be/src/runtime/coordinator.cc
File be/src/runtime/coordinator.cc:

http://gerrit.cloudera.org:8080/#/c/14824/26/be/src/runtime/coordinator.cc@511
PS26, Line 511: if (!failed_backend_states.empty()) {
Seems like it shouldn't be possible for this to ever be false. Maybe add a DCHECK?


http://gerrit.cloudera.org:8080/#/c/14824/26/be/src/runtime/coordinator.cc@512
PS26, Line 512: RETURN_IF_ERROR
I don't think its possible for HandleFailedExecRpcs() to return an error, and even if it did it probably wouldn't be correct to just return like this here since we still need to call UpdateExecState() below to update the query to an error status.


http://gerrit.cloudera.org:8080/#/c/14824/26/be/src/runtime/coordinator.cc@982
PS26, Line 982:       parent_query_driver_->TryQueryRetry(parent_request_state_, &retryable_status);
Probably fine to leave as is for now since currently any query that hits this is going to end up failing, but just wanted to point out that this code path means that we could potentially cancel the query without having actually received an error status back yet.

As noted elsewhere, this makes it more difficult to use the AddDetail() on the Status out parameter in TryQueryRetry.


http://gerrit.cloudera.org:8080/#/c/14824/26/be/src/runtime/coordinator.cc@1079
PS26, Line 1079:   parent_query_driver_->TryQueryRetry(parent_request_state_, &retryable_status);
Wanted to point out that there's an unfortunate situation here and elsewhere TryQueryRetry is called where we end up with two slightly different versions for the overall query status - the one that gets set on Coordinator (in this case, exec_rpc_status, which represents the error from the first backend to fail in Exec()) and 'retryable_status' which gets set on ClientRequestState in MarkAsRetrying().

Normally, those two statuses would be the same, because ClientRequestState would get Coordinator's state returned to it from Exec(), which it would then set on itself, it won't set it anymore as we only preserve the first error message. I think that's messy and has the potential to be confusing for developers.

Personally, I still really think it would be worth it to do the work to at least somewhat remove the circular dependency here, eg. you could move the call to TryQueryRetry to ClientRequestState::UpdateQueryStatus() along with defining a Coordinator::BlacklistedExecutors() or something like that, It wouldn't be much work, it would solve this problem, and it would make the control flow a lot cleaner, make Coordinator better encapsulated since it wouldn't need to know anything about retries, etc.


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h
File be/src/runtime/query-driver.h:

http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h@136
PS24, Line 136:   /// query option 'retry_failed_queries') and if the query can be retried. Queries can
> Doesn't return a Status anymore.
Sure, 'status' is used in one code path that calls TryQueryRetry (the code path in Coordinator where we get an error in a status report, which test_multiple_retries is testing), but its not used in any of the other code paths that call TryQueryRetry so it gets lost in those cases (eg. HandleFailedExecRpcs)

One option would be to just fix those code paths to actually set the status they passed in as the out parameter to the overall error status, but that's not always going to be easy (eg. the path in Coordinator where we get an AuxErrorInfo but no overall query error status in the report, so we have no error status to add the details to).

Instead, maybe it would be better to just add an info string to the profile that says "This query was not retried because..."

One disadvantage of that is that it means you can't immediately tell from the overall error why the query wasn't retried and have to dig down into the profile/logs, but that's already the case since you're only adding the detail if we hit the max retries and not for other cases such as if rows had already been fetched, though maybe you want to do an AddDetail for those cases too.


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h@204
PS24, Line 204:   /// 'RunFrontendPlanner'.
> Do you mean that CreateRetriedClientRequestState could just pass in a point
Personally, I think the code is way more confusing this way.

You're not "moving" the TExecRequest from the original ClientRequestState to the new one - the TExecRequest is just owned by the QueryDriver, so you're moving it from one place in the QueryDriver to a different place in the QueryDriver.

It doesn't change at any point, but if I was coming along reading this header file without the context of this patch, I would assume that since there's two separate TExecRequest variables they must be different.

It also unnecessarily complicates the object lifecycle, since now which of the two unique_ptr variables is valid changes depending on where in execution we are.


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-hs2-server.cc
File be/src/service/impala-hs2-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-hs2-server.cc@884
PS24, Line 884:   QueryHandle query_handle;
> I think the point of the QueryHandle is to enforce that whenever users use 
I understand that the point of QueryHandle is to make it harder for developers to screw up and let the shared_ptr go out of scope while still using the ClientRequestState.

My point is that as written it doesn't really do anything to accomplish that.

Taking this function as an example - there's nothing to stop a developer from letting 'query_handle' go out of scope while still using 'request_state'. For a developer that's just looking at this method, there's nothing to even hint that's a requirement.

In fact, the code is exactly identical to the code without QueryHandle except just that you replace 'shared_ptr<QueryDriver>' with QueryHandle and the declaration of 'request_state' from 'query_driver->GetRequestState()' to 'query_handle.request_state'. Again, all you've accomplished is that now instead of the rule being "you must remember to keep the shared_ptr on the stack" its "you must remember to keep the QueryHandle on the stack", so you haven't changed anything.

If you used my suggestion - make 'request_state' private in QueryHandle and redirect calls on QueryHandle to it with "operator->" the same way that unique_ptr works, then for a developer to call any functions on the ClientRequestState they have to actually have the QueryHandle object itself, thus actually ensuring that the shared_ptr hasn't gone out of scope.


http://gerrit.cloudera.org:8080/#/c/14824/26/common/thrift/generate_error_codes.py
File common/thrift/generate_error_codes.py:

http://gerrit.cloudera.org:8080/#/c/14824/26/common/thrift/generate_error_codes.py@467
PS26, Line 467:   ("RETRYABLE_ERROR", 151, "Retryable error: $0"),
I'm not certain this is the right thing to do.

It doesn't really seem consistent with how these error codes are used - all of the other codes (with maybe the exception of RECOVERABLE_ERROR) say what caused the error, whereas 'retryable' is a property that many different errors with different causes can have.

If you go this route, is there now an expectation that all errors that are potentially retryable use this? Does that prevent us from using more specific error codes to differentiate between different errors that are retryable?

The main effect its having in this patch is just to prepend "Retryable error:" to some error messages, which makes this less transparent, and I'm not sure how helpful that is anyways. To find out if it was in fact retried, if not why, etc. users will generally still have to go dig around in logs/the profile. Seems like it has more potential to just confuse users than to be helpful.

You also presumably don't want to return RETRYABLE_ERRORs to users that don't have retries turned on, but that means that almost identical query executions can return different error codes for the same error depending on a query option, which seems confusing.

I think you may also find this harder to use without doing things like returning RETRYABLE_ERRORs to users that don't even have retries on (which would definitely be confusing) once you go address my comments about losing the AddDetail from TryQueryRetry



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 26
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Mon, 11 May 2020 22:03:59 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Thomas Tauber-Marshall (Code Review)" <ge...@cloudera.org>.
Thomas Tauber-Marshall has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 24:

(16 comments)

http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/coordinator.cc
File be/src/runtime/coordinator.cc:

http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/coordinator.cc@937
PS24, Line 937: RETURN_IF_ERROR(
This isn't valid. (and you have similar issues elsewhere)

As far as I can tell, the only way that this returns is if we decide to retry the query and then Thread::Create returns an error, but even in that case we need to still finish this function and eg. call UpdateExecState below with the original error status.

In general, errors related to failing to retry aren't ever going to be overall query errors, and we'll always need to finish whatever processing we were doing for the original query, so probably all we need to do is log any errors from TryQueryRetry and move on. It may even be best to log the errors from within TryQueryRetry and not have it even return anything.


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h
File be/src/runtime/query-driver.h:

http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h@1
PS24, Line 1: // Licensed to the Apache Software Foundation (ASF) under one
The distinction isn't that clear, but it might make more sense to put this in /be/src/service, since right now its: 

impala-server (src/service) -> query-driver (src/runtime) -> client-request-state (src/service) -> coordinator (src/runtime)


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h@39
PS24, Line 39: QueryHandle
I think this name is confusing, esp. since there's already a QueryHandle in beeswax. Its really more of a ClientRequestStateHandle, though that's wordy. Maybe CRSHandle? ClientRequestHandle?

It might even be worth renaming ClientRequestState, since if anything the QueryDriver is what really holds the current state of the client request. Maybe QueryInstance? Not a huge deal though


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h@82
PS24, Line 82: /// *Transparent* Query Retries:
Not a huge deal in this patch since it's still hidden behind a flag and experimental, but I think that we should be cautious about saying that this is completely transparent, since it does change what clients see, potentially in ways that could break existing clients (eg. if you call GetRuntimeProfile() and then do an assert that the returned profile has the same query_id as what you requested, turning this on could cause that assert to fail).

Obviously at a minimum we need to test this with all the usual known clients (impala-shell, impyla, jdbc/odbc, Hue, etc.), and we'll probably be making changes to at least some of those clients for the sake of observability around this (eg. impala-shell might want to print a message about the query getting retried)


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h@136
PS24, Line 136:   Status TryQueryRetry(ClientRequestState* client_request_state, Status* status,
The way you're both returning a Status and also returning some status information in a Status out parameter seems confusing, and in fact it looks like at most of the call sites of this function the Status out parameter is never actually referenced again after the call so any info added to it is usually dropped.

I think its probably better to just not have 'status' be an out parameter and return use the returned status for everything, or possibly not return any status info at all (see my other comments)


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h@189
PS24, Line 189: A shared_ptr is used to allow asynchronous deletion
?


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h@194
PS24, Line 194: A shared_ptr is used to allow
              :   /// asynchronous deletion.
?


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.h@204
PS24, Line 204:   std::unique_ptr<TExecRequest> retry_exec_request_;
Now that this is just move()-ed from 'exec_request_' I'm not sure why its still needed.


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.cc
File be/src/runtime/query-driver.cc:

http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.cc@97
PS24, Line 97:     if (!client_request_state->fetched_rows()) {
This function would be more readable if you turned these into:

if (reason_not_to_retry) {
LOG << reason;
return;
}
if (another_reason) {
...

(after doing that, I personally would get rid of RetryAsync() as 1) its only called in one place 2) a lot of its code is just DCHECK-ing things that the ifs here enforce, so that could just be removed and 3) the names TryRetryQuery, RetryAsync, and RetryQueryFromThread get kind of confusing, but up to you)


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/runtime/query-driver.cc@302
PS24, Line 302: make_unique
Instead of this make_unique/move() thing, why not just have SetOriginalId() take a 'const TUniqueId&' and set the unique_ptr itself?


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-hs2-server.cc
File be/src/service/impala-hs2-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-hs2-server.cc@884
PS24, Line 884:   QueryHandle query_handle;
I'm not sure that in its current state QueryHandle is really doing much. This is all almost exactly equivalent to if this was just a bare shared_ptr<QueryDriver>, you've just moved the requirement from "developers must remember to keep the shared_ptr<QueryDriver> on the stack while using the ClientRequestState*" to "developers must remember to keep the QueryHandle on the stack...".

What if you did something like made the ClientRequestState* in QueryHandle private, and then implemented "operator->" for QueryHandle to effectively have it redirect any calls to the ClientRequestState*?

Would also be great to find a way to eliminate the GetClientRequestState/GetActiveClientRequestState functions entirely, to really force use of QueryHandle, but might be difficult without introducing an extra copy (and therefore atomic ref count inc/dec) of the shared_ptr


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-http-handler.cc
File be/src/service/impala-http-handler.cc:

http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-http-handler.cc@809
PS24, Line 809:   shared_ptr<QueryDriver> query_driver = server_->GetQueryDriver(query_id);
Noted elsewhere, but I think if you added an 'active' flag to GetQueryHandle() or something you could handle situations like these with QueryHandle as well.


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-server.h
File be/src/service/impala-server.h:

http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-server.h@932
PS24, Line 932:   Status GetQueryHandle(
Might be worth naming this GetActiveQueryHandle() for clarity.

Would also be good to define a regular GetQueryHandle() and/or add an 'active' flag to this function - I think if you did that, you could get rid of most of the remaining calls to GetQueryDriver() and use QueryHandle for everything.


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-server.cc
File be/src/service/impala-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-server.cc@1160
PS24, Line 1160:   RETURN_IF_ERROR(query_handle.query_driver->StartUnregister());
I'm not sure why this is necessary. I guess it has the effect of making UnregisterQuery() idempotent and thread safe, but wouldn't that already need to be the case, eg. because you could have a client that calls CloseImpalaOperation() twice concurrently? (maybe it wasn't?)

It also has the effect of making it look to clients like the query no longer exists synchronously with Unregister() instead of having to wait for FinishUnregisterQuery(), but I'm not sure why that would be needed for this patch if the previous behavior was correct (maybe it wasn't?)


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-server.cc@1167
PS24, Line 1167:   unreg_thread_pool_->Offer(query_handle);
I think you'll want to still call move() in situations like this - otherwise you're copying the shared_ptr<QueryDriver>, which results in an atomic inc/dec to the ref counter.


http://gerrit.cloudera.org:8080/#/c/14824/24/be/src/service/impala-server.cc@1586
PS24, Line 1586:         UnregisterQueryDiscardResult(query_id, true, &retry_status);
I'm wondering why we unregister the query in this case, but only cancel it in the other case where we decided not to retry. (I think we don't want to unregister here because we want the query to still be inspectable by the client, since we're the ones that decided to cancel it)



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 24
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Thu, 07 May 2020 21:25:43 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Joe McDonnell, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#29).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes (IMPALA-9124).

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the removed node, queries are retried), or (2) if a
query fails and as a result, blacklists a node. Either event is
considered a cluster membership change as it affects what nodes a query
will be scheduled on. The assumption is that a retry of the query with
the updated cluster membership will succeed.

A query retry is modelled as a brand new query, with its own query id.
This simplifies the implementation and the resulting runtime profiles
when queries are retried.

Core Features:
* Retries are transparent to the user; no modification to client
  libraries are necessary to support query retries
* Retried queries skip all fe/ parsing, planning, authorization, etc.
* Retries are configurable ('retry_failed_queries') and are off by
  default

Implementation:
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A new layer of abstraction between the ImpalaServer and
  ClientRequestState has been added; it is called the QueryDriver
* Each ClientRequestState is treated as a single attempt of a query, and
  the QueryDriver owns all ClientRequestStates for a query
* ClientRequestState has a new state object called RetryState; a
  ClientRequestState can either be NOT_RETRIED, RETRYING, or RETRIED
* The QueryDriver owns the TExecRequest for the query as well, it is
  re-used for each query retry

Observability:
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
* Runtime profiles of queries that fail and then are retried will have:
    * "Retry Status: RETRIED"
    * "Retry Cause: [the error that triggered the retry]"
    * "Retried Query Id: [the query id of the retried query]"
* Runtime profiles of the retried query (e.g. the second attempt of the
  query) will include:
    * "Original Query Id: [the query id of the original query]"
* The Impala Web UI will list all retried queries as being in the
  "RETRIED" state

Testing:
* Added E2E tests in test_query_retries.py; looped tests for a few days
* Added a stress test query_retries_stress_runner.py that runs concurrent
  streams of a TPC-{H,DS} workload and randomly kills impalads
* Ran the stress test with various configurations: tpch on parquet,
  tpcds on parquet, tpch 30 GB on parquet (one stream), tpcds 30 GB on
  parquet (one stream), tpch on text, tpcds on text
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures
* Ran 30 GB TPC-DS workload on a 3 node cluster, randomly restarted
  impalads, and manually verified that queries were retried
* Manually tested retries work with various clients, specifically the
  impala-shell and Hue
* Ran core tests and query retry stress test against an ASAN build
* Ran concurrent_select.py to stress query cancellation
* Ran be/ tests against a TSAN build, filed IMPALA-9730 as a follow up

Limitations:
* There are several limitations that are listed out in the parent JIRA

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/benchmarks/process-wide-locks-benchmark.cc
M be/src/runtime/CMakeLists.txt
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
A be/src/runtime/query-driver.cc
A be/src/runtime/query-driver.h
M be/src/service/CMakeLists.txt
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/control-service.cc
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
R be/src/service/query-driver-map.cc
A be/src/service/query-driver-map.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/testutil/impalad-query-executor.cc
M be/src/testutil/impalad-query-executor.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M tests/common/impala_cluster.py
M tests/common/impala_service.py
A tests/custom_cluster/test_query_retries.py
A tests/stress/query_retries_stress_runner.py
27 files changed, 2,703 insertions(+), 696 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/29
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 29
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 14:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/14824/14/be/src/runtime/query-driver.cc
File be/src/runtime/query-driver.cc:

http://gerrit.cloudera.org:8080/#/c/14824/14/be/src/runtime/query-driver.cc@163
PS14, Line 163:       &QueryDriver::RetryQueryFromThread, this, error, query_driver, &retry_query_thread_));
line too long (92 > 90)



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 14
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 01 May 2020 21:32:37 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 8:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/14824/8/tests/stress/query_retries_stress_runner.py
File tests/stress/query_retries_stress_runner.py:

http://gerrit.cloudera.org:8080/#/c/14824/8/tests/stress/query_retries_stress_runner.py@160
PS8, Line 160: e
flake8: E722 do not use bare except'


http://gerrit.cloudera.org:8080/#/c/14824/8/tests/stress/query_retries_stress_runner.py@164
PS8, Line 164: e
flake8: E722 do not use bare except'



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 8
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Tue, 31 Mar 2020 18:05:52 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#4).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes.

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the leaving node, queries are retried), and (2) if a
query fails and as a result, blacklists a node. Both events are
considered cluster membership changes as they affect what nodes a query
will be scheduled on.

Implementation:
* Query retries are driven by a dedicated threadpool
    * ImpalaServer::RetryQueryFromThreadPool implements the core logic to
      actually retry a failed query.
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A query cannot be retried once any results from the original query
  have been fetched, this is to prevent users from seeing incorrect results

Features:
* Retries are transparent to the user
    * This is achieved by adding a mapping from failed query ids to the
      query id of the retried query
    * ImpalaServer uses this mapping in GetClientFacingRequestState
      which is used to differentiate between "client facing" requests
      for a ClientRequestState vs. internal requrets for a CRS
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
    * "Impala Query Status" is a new field in runtime profiles that
      displays the ClientRequestState execution state (which includes
      the RETRYING and RETRIED states)
    * The Impala Web UI will list all retried queries as being in the
      "RETRIED" state
* Retried queries skip all fe/ planning, authorization, etc.
* This feature is configurable ('retry_failed_queries') and is off by
  default

Refactoring:
* Changes the ClientRequestState so that it can take in an existing
  TExecRequest
    * This is required when retrying queries because the
      TExecRequest of the failed query is copied and used for the
      ClientRequestState of the retried query
* ClientRequestState::ExecState is extended with three new states:
  RETRYING, RETRIED, and UNKNOWN.

Testing:
* Added integration tests in test_query_retries.py, these tests
  consistently pass when run locally
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures

TODO:
* There are some failed tests I am working through
* Additional re-factoring / code cleanup
* Lots more documentation

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
M be/src/service/client-request-state-map.h
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
A be/src/service/retry-work.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
A tests/custom_cluster/test_query_retries.py
16 files changed, 811 insertions(+), 166 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/4
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 4
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Joe McDonnell, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#11).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes (IMPALA-9124).

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the removed node, queries are retried), or (2) if a
query fails and as a result, blacklists a node. Either event is
considered a cluster membership change as it affects what nodes a query
will be scheduled on. The assumption is that a retry of the query with
the updated cluster membership will succeed.

A query retry is modelled as a brand new query, with its own query id.
This simplifies the implementation and the resulting runtime profiles
when queries are retried.

Core Features:
* Retries are transparent to the user; no modification to client
  libraries are necessary to support query retries
* Retried queries skip all fe/ parsing, planning, authorization, etc.
* Retries are configurable ('retry_failed_queries') and are off by
  default

Implementation:
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A new layer of abstraction between the ImpalaServer and
  ClientRequestState has been added; it is called the QueryDriver
* Each ClientRequestState is treated as a single attempt of a query, and
  the QueryDriver owns all ClientRequestStates for a query
* ClientRequestState has a new state object called RetryState; a
  ClientRequestState can either be NOT_RETRIED, RETRYING, or RETRIED
* The QueryDriver owns the TExecRequest for the query as well, it is
  re-used for each query retry

Observability:
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
* Runtime profiles of queries that fail and then are retried will have:
    * "Retry Status: RETRIED"
    * "Retry Cause: [the error that triggered the retry]"
    * "Retried Query Id: [the query id of the retried query]"
* Runtime profiles of the retried query (e.g. the second attempt of the
  query) will include:
    * "Original Query Id: [the query id of the original query]"
* The Impala Web UI will list all retried queries as being in the
  "RETRIED" state

Testing:
* Added E2E tests in test_query_retries.py
* Added a stress test query_retries_stress_runner.py that runs concurrent
  streams of a TPC workload and randomly kills impalads
* Ran the stress test with various configurations: tpch on parquet,
  tpcds on parquet, tpch 30 GB on parquet (one stream), tpcds 30 GB on
  parquet (one stream), tpch on text, tpcds on text
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures
* Ran 30 GB TPC-DS workload on a 3 node cluster, randomly restarted
  impalads, and manually verified that queries were retried
* Manually tested retries work with various clients, specifically the
  impala-shell and Hue

Limitations:
* IMPALA-9225: A query cannot be retried once any results from the original
  query have been fetched
* IMPALA-9200: A query can currently only be retried once

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/runtime/CMakeLists.txt
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
A be/src/runtime/query-driver.cc
A be/src/runtime/query-driver.h
M be/src/service/CMakeLists.txt
D be/src/service/client-request-state-map.h
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/control-service.cc
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
R be/src/service/query-driver-map.cc
A be/src/service/query-driver-map.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M common/thrift/generate_error_codes.py
M tests/common/impala_cluster.py
M tests/common/impala_service.py
A tests/custom_cluster/test_query_retries.py
A tests/stress/query_retries_stress_runner.py
26 files changed, 2,258 insertions(+), 433 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/11
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 11
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 17:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/5970/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 17
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Wed, 06 May 2020 02:09:40 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Joe McDonnell, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#27).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes (IMPALA-9124).

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the removed node, queries are retried), or (2) if a
query fails and as a result, blacklists a node. Either event is
considered a cluster membership change as it affects what nodes a query
will be scheduled on. The assumption is that a retry of the query with
the updated cluster membership will succeed.

A query retry is modelled as a brand new query, with its own query id.
This simplifies the implementation and the resulting runtime profiles
when queries are retried.

Core Features:
* Retries are transparent to the user; no modification to client
  libraries are necessary to support query retries
* Retried queries skip all fe/ parsing, planning, authorization, etc.
* Retries are configurable ('retry_failed_queries') and are off by
  default

Implementation:
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A new layer of abstraction between the ImpalaServer and
  ClientRequestState has been added; it is called the QueryDriver
* Each ClientRequestState is treated as a single attempt of a query, and
  the QueryDriver owns all ClientRequestStates for a query
* ClientRequestState has a new state object called RetryState; a
  ClientRequestState can either be NOT_RETRIED, RETRYING, or RETRIED
* The QueryDriver owns the TExecRequest for the query as well, it is
  re-used for each query retry

Observability:
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
* Runtime profiles of queries that fail and then are retried will have:
    * "Retry Status: RETRIED"
    * "Retry Cause: [the error that triggered the retry]"
    * "Retried Query Id: [the query id of the retried query]"
* Runtime profiles of the retried query (e.g. the second attempt of the
  query) will include:
    * "Original Query Id: [the query id of the original query]"
* The Impala Web UI will list all retried queries as being in the
  "RETRIED" state

Testing:
* Added E2E tests in test_query_retries.py; looped tests for a few days
* Added a stress test query_retries_stress_runner.py that runs concurrent
  streams of a TPC-{H,DS} workload and randomly kills impalads
* Ran the stress test with various configurations: tpch on parquet,
  tpcds on parquet, tpch 30 GB on parquet (one stream), tpcds 30 GB on
  parquet (one stream), tpch on text, tpcds on text
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures
* Ran 30 GB TPC-DS workload on a 3 node cluster, randomly restarted
  impalads, and manually verified that queries were retried
* Manually tested retries work with various clients, specifically the
  impala-shell and Hue
* Ran core tests and query retry stress test against an ASAN build
* Ran concurrent_select.py to stress query cancellation
* Ran be/ tests against a TSAN build, filed IMPALA-9730 as a follow up

Limitations:
* There are several limitations that are listed out in the parent JIRA

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/benchmarks/process-wide-locks-benchmark.cc
M be/src/runtime/CMakeLists.txt
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
A be/src/runtime/query-driver.cc
A be/src/runtime/query-driver.h
M be/src/service/CMakeLists.txt
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/control-service.cc
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
R be/src/service/query-driver-map.cc
A be/src/service/query-driver-map.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/testutil/impalad-query-executor.cc
M be/src/testutil/impalad-query-executor.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M tests/common/impala_cluster.py
M tests/common/impala_service.py
A tests/custom_cluster/test_query_retries.py
A tests/stress/query_retries_stress_runner.py
27 files changed, 2,634 insertions(+), 675 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/27
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 27
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 30:

Build started: https://jenkins.impala.io/job/gerrit-verify-dryrun/5836/ DRY_RUN=true


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 30
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Thu, 14 May 2020 20:56:52 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Joe McDonnell (Code Review)" <ge...@cloudera.org>.
Joe McDonnell has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 9:

(7 comments)

http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/runtime/coordinator.cc
File be/src/runtime/coordinator.cc:

http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/runtime/coordinator.cc@502
PS9, Line 502:         RETURN_IF_ERROR(HandleFailedExecRpc(backend_state));
For this loop, do we have any expectation about the number of BackendStates that have errors? In other words, can more than one have errors?

If there can be more than one error, we would call TryQueryRetry more than once, but based on my reading of that function, it can handle that. Am I right?

Is there any concern that if there is more than one error, we may want to blacklist more than one Impalad before retrying?


http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/runtime/query-driver.h
File be/src/runtime/query-driver.h:

http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/runtime/query-driver.h@116
PS9, Line 116: a
             :   /// statestore updates
I think this should be singular "a statestore update"


http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/runtime/query-driver.h@122
PS9, Line 122:   Status TryQueryRetry(ClientRequestState* client_request_state, Status* status,
             :       bool* was_retried = nullptr) WARN_UNUSED_RESULT;
From reading through TryQueryRetry(), it looks like this is safe to call multiple times and it will only do the retry once. Is that true? Can we add a comment here that spells out that it can be called more than once?


http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/service/impala-hs2-server.cc
File be/src/service/impala-hs2-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/service/impala-hs2-server.cc@763
PS9, Line 763:   ClientRequestState* request_state = nullptr;
> So I'm wondering if its safe to no longer have any shared_ptr here and in t
I had the same concern.


http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/service/impala-server.h
File be/src/service/impala-server.h:

http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/service/impala-server.h@673
PS9, Line 673: ClientRequestState*&
Stylistically, I personally don't like mutable references for out arguments. Using ** instead makes it much clearer that values are being passed between functions. Callers need to pass addresses. The function itself can't treat it like a local variable.


http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/service/impala-server.cc
File be/src/service/impala-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/service/impala-server.cc@1453
PS9, Line 1453:   ClientRequestState::RetryState retry_state;
Nit: you could move this declaration down past the first BlockOnWait()


http://gerrit.cloudera.org:8080/#/c/14824/9/tests/custom_cluster/test_query_retries.py
File tests/custom_cluster/test_query_retries.py:

http://gerrit.cloudera.org:8080/#/c/14824/9/tests/custom_cluster/test_query_retries.py@93
PS9, Line 93: Increase the statestore
            :     heartbeat frequency so that the query actually fails during execution.
It looks like we are using the default value for statestore_heartbeat_frequency_ms. Is that intentional?

If I understand this correctly, this test uses the heavy shuffle query and should see an RPC failure. We are not expecting the statestore to see the Impalad as dead. Is that right?



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 9
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Mon, 27 Apr 2020 17:48:11 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 6:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/14824/6/be/src/service/impala-server.cc
File be/src/service/impala-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/6/be/src/service/impala-server.cc@1463
PS6, Line 1463: void ImpalaServer::BlockOnWait(shared_ptr<ClientRequestState>* request_state, bool* timed_out,
line too long (94 > 90)



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 6
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Mon, 10 Feb 2020 16:51:03 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 3:

Build Failed 

https://jenkins.impala.io/job/gerrit-code-review-checks/5515/ : Initial code review checks failed. See linked job for details on the failure.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 3
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 24 Jan 2020 17:41:36 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 14:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/5927/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 14
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 01 May 2020 22:16:10 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 9:

After a lot of cleanup, doc updates, and testing, I think this patch is finally ready to go (open to general comments). I made a big change to the commit message to describe the patch + all the testing I have done. There are a few TODOs sprinkled around the code, but non are essential (most are cleanup related).


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 9
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Tue, 31 Mar 2020 18:45:21 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 9:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/14824/13/be/src/runtime/query-driver.h
File be/src/runtime/query-driver.h:

http://gerrit.cloudera.org:8080/#/c/14824/13/be/src/runtime/query-driver.h@169
PS13, Line 169: _state_;
> I should have mentioned that the shared_ptr approach also requires special 
i decided just to create a struct called QueryHandle that contains a shared_ptr to the QueryDriver and a naked pointer to the ClientRequestState. I reviewed all the places where a QueryDriver / ClientRequestState are used and make sure the ClientRequestState is only used when the QueryDriver is still in scope. I used new QueryDriver struct where I thought it was useful (e.g. ImpalaServer::GetQueryHandle).
i added some docs that explain the ClientRequestState should only be used when a valid QueryDriver is in scope.
since the QueryDriver must be in scope while the ClientRequestState is being used, I changed the ownership of the ClientRequestState back to unique_ptr and changed the Get*ClientRequestState() methods to return a naked pointer.


http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/service/impala-hs2-server.cc
File be/src/service/impala-hs2-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/service/impala-hs2-server.cc@763
PS9, Line 763:   ClientRequestState* request_state = nullptr;
> yeah, thats a good point, thanks for catching that. changed it to use a sha
i ended it up changing it back to unique_ptr. I changed the semantics a bit so that whenever you use a ClientRequestState, the corresponding QueryDriver shared_ptr must have an active reference on the stack. I added a new struct called QueryHandle that contains a shared_ptr to a QueryDriver and a naked pointer to a ClientRequestState. the idea is that the QueryHandle will help insure a QueryDriver reference is alive when accessing a ClientRequestState.
there are a few places where it didn't make sense to use the new QueryHandle, so I just spot checked that whenever a ClientRequestState is used, the corresponding QueryDriver is still in scope.



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 9
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Mon, 04 May 2020 01:06:31 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 5:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/14824/5/be/src/service/impala-server.cc
File be/src/service/impala-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/5/be/src/service/impala-server.cc@1025
PS5, Line 1025: query_ctx->query_id = UuidToQueryId(random_generator()());
> I sort of did this in the previous patch update. I added a client_query_id_
I'll try thinking through this some more. One challenge with creating an internal vs. user facing id is that a common debugging practice is to take user facing id and use it to grep through the logs. A lot of the log statements contain the query id they are associated with. So the user would have to manually figure out the mapping from the user facing id (perhaps it got the id from the runtime profile or the client logs) and the internal id.



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 5
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Thu, 13 Feb 2020 19:06:57 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 13:

(2 comments)

addressed remaining comments

http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/runtime/coordinator.cc
File be/src/runtime/coordinator.cc:

http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/runtime/coordinator.cc@502
PS9, Line 502:           && !backend_state->exec_rpc_status().IsAborted()) 
> For this loop, do we have any expectation about the number of BackendStates
yeah, good point. fixed.


http://gerrit.cloudera.org:8080/#/c/14824/9/tests/custom_cluster/test_query_retries.py
File tests/custom_cluster/test_query_retries.py:

http://gerrit.cloudera.org:8080/#/c/14824/9/tests/custom_cluster/test_query_retries.py@93
PS9, Line 93: Increase the statestore
            :     heartbeat frequency so that the query actually fails during execution.
> It looks like we are using the default value for statestore_heartbeat_frequ
yeah - the tests infra actually sets a lower value for this: https://github.com/apache/impala/blob/master/tests/common/custom_cluster_test_suite.py#L59

yes, your understanding is correct. a RPC should fail which should cause the query to be retried.



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 13
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Thu, 30 Apr 2020 12:53:15 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Joe McDonnell, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#14).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes (IMPALA-9124).

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the removed node, queries are retried), or (2) if a
query fails and as a result, blacklists a node. Either event is
considered a cluster membership change as it affects what nodes a query
will be scheduled on. The assumption is that a retry of the query with
the updated cluster membership will succeed.

A query retry is modelled as a brand new query, with its own query id.
This simplifies the implementation and the resulting runtime profiles
when queries are retried.

Core Features:
* Retries are transparent to the user; no modification to client
  libraries are necessary to support query retries
* Retried queries skip all fe/ parsing, planning, authorization, etc.
* Retries are configurable ('retry_failed_queries') and are off by
  default

Implementation:
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A new layer of abstraction between the ImpalaServer and
  ClientRequestState has been added; it is called the QueryDriver
* Each ClientRequestState is treated as a single attempt of a query, and
  the QueryDriver owns all ClientRequestStates for a query
* ClientRequestState has a new state object called RetryState; a
  ClientRequestState can either be NOT_RETRIED, RETRYING, or RETRIED
* The QueryDriver owns the TExecRequest for the query as well, it is
  re-used for each query retry

Observability:
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
* Runtime profiles of queries that fail and then are retried will have:
    * "Retry Status: RETRIED"
    * "Retry Cause: [the error that triggered the retry]"
    * "Retried Query Id: [the query id of the retried query]"
* Runtime profiles of the retried query (e.g. the second attempt of the
  query) will include:
    * "Original Query Id: [the query id of the original query]"
* The Impala Web UI will list all retried queries as being in the
  "RETRIED" state

Testing:
* Added E2E tests in test_query_retries.py
* Added a stress test query_retries_stress_runner.py that runs concurrent
  streams of a TPC workload and randomly kills impalads
* Ran the stress test with various configurations: tpch on parquet,
  tpcds on parquet, tpch 30 GB on parquet (one stream), tpcds 30 GB on
  parquet (one stream), tpch on text, tpcds on text
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures
* Ran 30 GB TPC-DS workload on a 3 node cluster, randomly restarted
  impalads, and manually verified that queries were retried
* Manually tested retries work with various clients, specifically the
  impala-shell and Hue

Limitations:
* There are several limitations that are listed out in the parent JIRA

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/runtime/CMakeLists.txt
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
A be/src/runtime/query-driver.cc
A be/src/runtime/query-driver.h
M be/src/service/CMakeLists.txt
D be/src/service/client-request-state-map.h
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/control-service.cc
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
R be/src/service/query-driver-map.cc
A be/src/service/query-driver-map.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M common/thrift/generate_error_codes.py
M tests/common/impala_cluster.py
M tests/common/impala_service.py
A tests/custom_cluster/test_query_retries.py
A tests/stress/query_retries_stress_runner.py
26 files changed, 2,300 insertions(+), 427 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/14
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 14
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 25:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/6011/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 25
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 08 May 2020 20:04:18 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Joe McDonnell, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#12).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes (IMPALA-9124).

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the removed node, queries are retried), or (2) if a
query fails and as a result, blacklists a node. Either event is
considered a cluster membership change as it affects what nodes a query
will be scheduled on. The assumption is that a retry of the query with
the updated cluster membership will succeed.

A query retry is modelled as a brand new query, with its own query id.
This simplifies the implementation and the resulting runtime profiles
when queries are retried.

Core Features:
* Retries are transparent to the user; no modification to client
  libraries are necessary to support query retries
* Retried queries skip all fe/ parsing, planning, authorization, etc.
* Retries are configurable ('retry_failed_queries') and are off by
  default

Implementation:
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A new layer of abstraction between the ImpalaServer and
  ClientRequestState has been added; it is called the QueryDriver
* Each ClientRequestState is treated as a single attempt of a query, and
  the QueryDriver owns all ClientRequestStates for a query
* ClientRequestState has a new state object called RetryState; a
  ClientRequestState can either be NOT_RETRIED, RETRYING, or RETRIED
* The QueryDriver owns the TExecRequest for the query as well, it is
  re-used for each query retry

Observability:
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
* Runtime profiles of queries that fail and then are retried will have:
    * "Retry Status: RETRIED"
    * "Retry Cause: [the error that triggered the retry]"
    * "Retried Query Id: [the query id of the retried query]"
* Runtime profiles of the retried query (e.g. the second attempt of the
  query) will include:
    * "Original Query Id: [the query id of the original query]"
* The Impala Web UI will list all retried queries as being in the
  "RETRIED" state

Testing:
* Added E2E tests in test_query_retries.py
* Added a stress test query_retries_stress_runner.py that runs concurrent
  streams of a TPC workload and randomly kills impalads
* Ran the stress test with various configurations: tpch on parquet,
  tpcds on parquet, tpch 30 GB on parquet (one stream), tpcds 30 GB on
  parquet (one stream), tpch on text, tpcds on text
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures
* Ran 30 GB TPC-DS workload on a 3 node cluster, randomly restarted
  impalads, and manually verified that queries were retried
* Manually tested retries work with various clients, specifically the
  impala-shell and Hue

Limitations:
* IMPALA-9225: A query cannot be retried once any results from the original
  query have been fetched
* IMPALA-9200: A query can currently only be retried once

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/runtime/CMakeLists.txt
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
A be/src/runtime/query-driver.cc
A be/src/runtime/query-driver.h
M be/src/service/CMakeLists.txt
D be/src/service/client-request-state-map.h
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/control-service.cc
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
R be/src/service/query-driver-map.cc
A be/src/service/query-driver-map.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M common/thrift/generate_error_codes.py
M tests/common/impala_cluster.py
M tests/common/impala_service.py
A tests/custom_cluster/test_query_retries.py
A tests/stress/query_retries_stress_runner.py
26 files changed, 2,259 insertions(+), 433 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/12
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 12
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Joe McDonnell, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#18).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes (IMPALA-9124).

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the removed node, queries are retried), or (2) if a
query fails and as a result, blacklists a node. Either event is
considered a cluster membership change as it affects what nodes a query
will be scheduled on. The assumption is that a retry of the query with
the updated cluster membership will succeed.

A query retry is modelled as a brand new query, with its own query id.
This simplifies the implementation and the resulting runtime profiles
when queries are retried.

Core Features:
* Retries are transparent to the user; no modification to client
  libraries are necessary to support query retries
* Retried queries skip all fe/ parsing, planning, authorization, etc.
* Retries are configurable ('retry_failed_queries') and are off by
  default

Implementation:
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A new layer of abstraction between the ImpalaServer and
  ClientRequestState has been added; it is called the QueryDriver
* Each ClientRequestState is treated as a single attempt of a query, and
  the QueryDriver owns all ClientRequestStates for a query
* ClientRequestState has a new state object called RetryState; a
  ClientRequestState can either be NOT_RETRIED, RETRYING, or RETRIED
* The QueryDriver owns the TExecRequest for the query as well, it is
  re-used for each query retry

Observability:
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
* Runtime profiles of queries that fail and then are retried will have:
    * "Retry Status: RETRIED"
    * "Retry Cause: [the error that triggered the retry]"
    * "Retried Query Id: [the query id of the retried query]"
* Runtime profiles of the retried query (e.g. the second attempt of the
  query) will include:
    * "Original Query Id: [the query id of the original query]"
* The Impala Web UI will list all retried queries as being in the
  "RETRIED" state

Testing:
* Added E2E tests in test_query_retries.py; looped tests for a few days
* Added a stress test query_retries_stress_runner.py that runs concurrent
  streams of a TPC workload and randomly kills impalads
* Ran the stress test with various configurations: tpch on parquet,
  tpcds on parquet, tpch 30 GB on parquet (one stream), tpcds 30 GB on
  parquet (one stream), tpch on text, tpcds on text
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures
* Ran 30 GB TPC-DS workload on a 3 node cluster, randomly restarted
  impalads, and manually verified that queries were retried
* Manually tested retries work with various clients, specifically the
  impala-shell and Hue
* Ran core tests and query retry stress test against an ASAN build
* Ran concurrent_select.py to stress query cancellation

Limitations:
* There are several limitations that are listed out in the parent JIRA

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/runtime/CMakeLists.txt
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
A be/src/runtime/query-driver.cc
A be/src/runtime/query-driver.h
M be/src/service/CMakeLists.txt
D be/src/service/client-request-state-map.h
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/control-service.cc
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
R be/src/service/query-driver-map.cc
A be/src/service/query-driver-map.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/testutil/impalad-query-executor.cc
M be/src/testutil/impalad-query-executor.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M common/thrift/generate_error_codes.py
M tests/common/impala_cluster.py
M tests/common/impala_service.py
A tests/custom_cluster/test_query_retries.py
A tests/stress/query_retries_stress_runner.py
28 files changed, 2,372 insertions(+), 515 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/18
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 18
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9124 [POC][WIP]: Transparently retry queries that fail due to cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9124 [POC][WIP]: Transparently retry queries that fail due to cluster membership changes
......................................................................


Patch Set 1:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/14824/1/be/src/service/client-request-state.cc
File be/src/service/client-request-state.cc:

http://gerrit.cloudera.org:8080/#/c/14824/1/be/src/service/client-request-state.cc@981
PS1, Line 981:       // cancellation thread pool can kill a query while it is in the INITIALIZATION phase.
line too long (91 > 90)


http://gerrit.cloudera.org:8080/#/c/14824/1/be/src/service/impala-server.h
File be/src/service/impala-server.h:

http://gerrit.cloudera.org:8080/#/c/14824/1/be/src/service/impala-server.h@993
PS1, Line 993:   /// order to avoid query compilation and planning again. Once the new query is registered
line too long (91 > 90)



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 1
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Michael Ho <mi...@gmail.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Wed, 11 Dec 2019 01:24:51 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Joe McDonnell, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#16).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes (IMPALA-9124).

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the removed node, queries are retried), or (2) if a
query fails and as a result, blacklists a node. Either event is
considered a cluster membership change as it affects what nodes a query
will be scheduled on. The assumption is that a retry of the query with
the updated cluster membership will succeed.

A query retry is modelled as a brand new query, with its own query id.
This simplifies the implementation and the resulting runtime profiles
when queries are retried.

Core Features:
* Retries are transparent to the user; no modification to client
  libraries are necessary to support query retries
* Retried queries skip all fe/ parsing, planning, authorization, etc.
* Retries are configurable ('retry_failed_queries') and are off by
  default

Implementation:
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A new layer of abstraction between the ImpalaServer and
  ClientRequestState has been added; it is called the QueryDriver
* Each ClientRequestState is treated as a single attempt of a query, and
  the QueryDriver owns all ClientRequestStates for a query
* ClientRequestState has a new state object called RetryState; a
  ClientRequestState can either be NOT_RETRIED, RETRYING, or RETRIED
* The QueryDriver owns the TExecRequest for the query as well, it is
  re-used for each query retry

Observability:
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
* Runtime profiles of queries that fail and then are retried will have:
    * "Retry Status: RETRIED"
    * "Retry Cause: [the error that triggered the retry]"
    * "Retried Query Id: [the query id of the retried query]"
* Runtime profiles of the retried query (e.g. the second attempt of the
  query) will include:
    * "Original Query Id: [the query id of the original query]"
* The Impala Web UI will list all retried queries as being in the
  "RETRIED" state

Testing:
* Added E2E tests in test_query_retries.py; looped tests for a few days
* Added a stress test query_retries_stress_runner.py that runs concurrent
  streams of a TPC workload and randomly kills impalads
* Ran the stress test with various configurations: tpch on parquet,
  tpcds on parquet, tpch 30 GB on parquet (one stream), tpcds 30 GB on
  parquet (one stream), tpch on text, tpcds on text
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures
* Ran 30 GB TPC-DS workload on a 3 node cluster, randomly restarted
  impalads, and manually verified that queries were retried
* Manually tested retries work with various clients, specifically the
  impala-shell and Hue
* Ran core tests and query retry stress test against an ASAN build
* Ran concurrent_select.py to stress query cancellation

Limitations:
* There are several limitations that are listed out in the parent JIRA

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/runtime/CMakeLists.txt
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
A be/src/runtime/query-driver.cc
A be/src/runtime/query-driver.h
M be/src/service/CMakeLists.txt
D be/src/service/client-request-state-map.h
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/control-service.cc
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
R be/src/service/query-driver-map.cc
A be/src/service/query-driver-map.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M common/thrift/generate_error_codes.py
M tests/common/impala_cluster.py
M tests/common/impala_service.py
A tests/custom_cluster/test_query_retries.py
A tests/stress/query_retries_stress_runner.py
26 files changed, 2,328 insertions(+), 473 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/16
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 16
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 16:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/5941/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 16
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Mon, 04 May 2020 01:52:24 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 32:

Build started: https://jenkins.impala.io/job/gerrit-verify-dryrun/5845/ DRY_RUN=true


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 32
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 15 May 2020 14:44:44 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Thomas Tauber-Marshall (Code Review)" <ge...@cloudera.org>.
Thomas Tauber-Marshall has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 13:

(4 comments)

Still going through it

http://gerrit.cloudera.org:8080/#/c/14824/13/be/src/runtime/query-driver.cc
File be/src/runtime/query-driver.cc:

http://gerrit.cloudera.org:8080/#/c/14824/13/be/src/runtime/query-driver.cc@161
PS13, Line 161: void QueryDriver::RetryQueryFromThread(const Status& error) {
Is there anything that prevents the QueryDriver from being deleted while this thread is running? eg. what if the query fails and then while the retry is happening the user cancels the query? Mostly we're relying on the shared_ptr to prevent those sorts of issues, but I don't think we're holding a shared_ptr for the duration of this, since you have to just pass a bare 'this' to Thread::Create


http://gerrit.cloudera.org:8080/#/c/14824/13/be/src/runtime/query-driver.cc@178
PS13, Line 178:   // Cancel the query.
Is this necessary? At least in the case of Coordinator calling TryQueryRetry() we'll end up calling cancel from the Coordinator too when the status is updated to an error. I guess its fine because Cancel() is idempotent, but might be worth thinking about what we can do to make this cleaner.


http://gerrit.cloudera.org:8080/#/c/14824/13/be/src/runtime/query-driver.cc@179
PS13, Line 179: true, nullptr
Both of these parameters seem wrong - you note above that a query could get retried when its still in the INITIALIZED state, so seems like 'check_inflight=true' could fail, and according to the comment about ClientRequestState::Cancel 'nullptr' for cause is supposed to indicate a user-initiated cancel that doesn't have an associated error, which this isn't.

I think in the case of an error that comes from the query itself (i.e. Coordinator call TryQueryRetry()) you'll get the error set anyways when UpdateExecState() is called (unless the AuxErrorInfo comes in a different rpc than the overall error message, in which case I think there's a race between the coordinator getting cancelled and no longer accepting reports, though I'm not sure), and in the case of a backend crashing (i.e. CancelFromThreadPool calls TryQueryRetry()), I don't think the error will get set.


http://gerrit.cloudera.org:8080/#/c/14824/13/be/src/runtime/query-driver.cc@281
PS13, Line 281:   // TODO: IMPALA-9502: Avoid copying TExecRequest when retrying queries
Out of curiosity, what are the issues here?



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 13
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Thu, 30 Apr 2020 23:04:08 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 4:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/14824/4/be/src/service/impala-server.h
File be/src/service/impala-server.h:

http://gerrit.cloudera.org:8080/#/c/14824/4/be/src/service/impala-server.h@1007
PS4, Line 1007:   /// order to avoid query compilation and planning again. Once the new query is registered
line too long (91 > 90)



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 4
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 24 Jan 2020 22:33:08 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Joe McDonnell, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#21).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes (IMPALA-9124).

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the removed node, queries are retried), or (2) if a
query fails and as a result, blacklists a node. Either event is
considered a cluster membership change as it affects what nodes a query
will be scheduled on. The assumption is that a retry of the query with
the updated cluster membership will succeed.

A query retry is modelled as a brand new query, with its own query id.
This simplifies the implementation and the resulting runtime profiles
when queries are retried.

Core Features:
* Retries are transparent to the user; no modification to client
  libraries are necessary to support query retries
* Retried queries skip all fe/ parsing, planning, authorization, etc.
* Retries are configurable ('retry_failed_queries') and are off by
  default

Implementation:
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A new layer of abstraction between the ImpalaServer and
  ClientRequestState has been added; it is called the QueryDriver
* Each ClientRequestState is treated as a single attempt of a query, and
  the QueryDriver owns all ClientRequestStates for a query
* ClientRequestState has a new state object called RetryState; a
  ClientRequestState can either be NOT_RETRIED, RETRYING, or RETRIED
* The QueryDriver owns the TExecRequest for the query as well, it is
  re-used for each query retry

Observability:
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
* Runtime profiles of queries that fail and then are retried will have:
    * "Retry Status: RETRIED"
    * "Retry Cause: [the error that triggered the retry]"
    * "Retried Query Id: [the query id of the retried query]"
* Runtime profiles of the retried query (e.g. the second attempt of the
  query) will include:
    * "Original Query Id: [the query id of the original query]"
* The Impala Web UI will list all retried queries as being in the
  "RETRIED" state

Testing:
* Added E2E tests in test_query_retries.py; looped tests for a few days
* Added a stress test query_retries_stress_runner.py that runs concurrent
  streams of a TPC workload and randomly kills impalads
* Ran the stress test with various configurations: tpch on parquet,
  tpcds on parquet, tpch 30 GB on parquet (one stream), tpcds 30 GB on
  parquet (one stream), tpch on text, tpcds on text
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures
* Ran 30 GB TPC-DS workload on a 3 node cluster, randomly restarted
  impalads, and manually verified that queries were retried
* Manually tested retries work with various clients, specifically the
  impala-shell and Hue
* Ran core tests and query retry stress test against an ASAN build
* Ran concurrent_select.py to stress query cancellation

Limitations:
* There are several limitations that are listed out in the parent JIRA

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/benchmarks/process-wide-locks-benchmark.cc
M be/src/runtime/CMakeLists.txt
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
A be/src/runtime/query-driver.cc
A be/src/runtime/query-driver.h
M be/src/service/CMakeLists.txt
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/control-service.cc
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
R be/src/service/query-driver-map.cc
A be/src/service/query-driver-map.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/testutil/impalad-query-executor.cc
M be/src/testutil/impalad-query-executor.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M common/thrift/generate_error_codes.py
M tests/common/impala_cluster.py
M tests/common/impala_service.py
A tests/custom_cluster/test_query_retries.py
A tests/stress/query_retries_stress_runner.py
28 files changed, 2,445 insertions(+), 510 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/21
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 21
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 15:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/5928/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 15
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 01 May 2020 22:17:58 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 19:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/14824/19/be/src/service/impala-server.cc
File be/src/service/impala-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/19/be/src/service/impala-server.cc@1153
PS19, Line 1153:   
> line has trailing whitespace
Done


http://gerrit.cloudera.org:8080/#/c/14824/19/be/src/service/impala-server.cc@1310
PS19, Line 1310:   RETURN_IF_ERROR(query_handle.request_state->Cancel(/*check_inflight=*/ true, /*cause=*/ nullptr));
> line too long (100 > 90)
Done



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 19
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Wed, 06 May 2020 03:12:35 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Joe McDonnell (Code Review)" <ge...@cloudera.org>.
Joe McDonnell has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 13:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/14824/13/be/src/runtime/query-driver.h
File be/src/runtime/query-driver.h:

http://gerrit.cloudera.org:8080/#/c/14824/13/be/src/runtime/query-driver.h@169
PS13, Line 169: 
ClientRequestState has a pointer back to the QueryDriver (as does the Coordinator), so if ClientRequestState outlives the QueryDriver, then that pointer would be dangling.

I think the simplest lifetime semantic is for the QueryDriver to live as long as any child ClientRequestStates is alive.

I think we are pretty close to that being true. Most code looks up a QueryDriver and then looks up the ClientRequestState with the QueryDriver shared_ptr staying on the stack. One option is to formalize it and have a struct that contains a shared_ptr to the QueryDriver and then a pointer to ClientRequestState. If everything uses that struct when operating on a ClientRequestState, then the QueryDriver will always outlive its children ClientRequestStates. This is just a sketch, so I might be missing something.

An implicit way to do this is to have a shared_ptr from ClientRequestState to the QueryDriver.



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 13
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 01 May 2020 17:48:05 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Joe McDonnell, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#15).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes (IMPALA-9124).

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the removed node, queries are retried), or (2) if a
query fails and as a result, blacklists a node. Either event is
considered a cluster membership change as it affects what nodes a query
will be scheduled on. The assumption is that a retry of the query with
the updated cluster membership will succeed.

A query retry is modelled as a brand new query, with its own query id.
This simplifies the implementation and the resulting runtime profiles
when queries are retried.

Core Features:
* Retries are transparent to the user; no modification to client
  libraries are necessary to support query retries
* Retried queries skip all fe/ parsing, planning, authorization, etc.
* Retries are configurable ('retry_failed_queries') and are off by
  default

Implementation:
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A new layer of abstraction between the ImpalaServer and
  ClientRequestState has been added; it is called the QueryDriver
* Each ClientRequestState is treated as a single attempt of a query, and
  the QueryDriver owns all ClientRequestStates for a query
* ClientRequestState has a new state object called RetryState; a
  ClientRequestState can either be NOT_RETRIED, RETRYING, or RETRIED
* The QueryDriver owns the TExecRequest for the query as well, it is
  re-used for each query retry

Observability:
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
* Runtime profiles of queries that fail and then are retried will have:
    * "Retry Status: RETRIED"
    * "Retry Cause: [the error that triggered the retry]"
    * "Retried Query Id: [the query id of the retried query]"
* Runtime profiles of the retried query (e.g. the second attempt of the
  query) will include:
    * "Original Query Id: [the query id of the original query]"
* The Impala Web UI will list all retried queries as being in the
  "RETRIED" state

Testing:
* Added E2E tests in test_query_retries.py
* Added a stress test query_retries_stress_runner.py that runs concurrent
  streams of a TPC workload and randomly kills impalads
* Ran the stress test with various configurations: tpch on parquet,
  tpcds on parquet, tpch 30 GB on parquet (one stream), tpcds 30 GB on
  parquet (one stream), tpch on text, tpcds on text
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures
* Ran 30 GB TPC-DS workload on a 3 node cluster, randomly restarted
  impalads, and manually verified that queries were retried
* Manually tested retries work with various clients, specifically the
  impala-shell and Hue

Limitations:
* There are several limitations that are listed out in the parent JIRA

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/runtime/CMakeLists.txt
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
A be/src/runtime/query-driver.cc
A be/src/runtime/query-driver.h
M be/src/service/CMakeLists.txt
D be/src/service/client-request-state-map.h
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/control-service.cc
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
R be/src/service/query-driver-map.cc
A be/src/service/query-driver-map.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M common/thrift/generate_error_codes.py
M tests/common/impala_cluster.py
M tests/common/impala_service.py
A tests/custom_cluster/test_query_retries.py
A tests/stress/query_retries_stress_runner.py
26 files changed, 2,301 insertions(+), 427 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/15
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 15
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 19:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/5972/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 19
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Wed, 06 May 2020 03:53:21 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#3).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes.

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the leaving node, queries are retried), and (2) if a
query fails and as a result, blacklists a node. Both events are
considered cluster membership changes as they affect what nodes a query
will be scheduled on.

Implementation:
* Query retries are driven by a dedicated threadpool
    * ImpalaServer::RetryQueryFromThreadPool implements the core logic to
      actually retry a failed query.
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A query cannot be retried once any results from the original query
  have been fetched, this is to prevent users from seeing incorrect results

Features:
* Retries are transparent to the user
    * This is achieved by adding a mapping from failed query ids to the
      query id of the retried query
    * ImpalaServer uses this mapping in GetClientFacingRequestState
      which is used to differentiate between "client facing" requests
      for a ClientRequestState vs. internal requrets for a CRS
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
    * "Impala Query Status" is a new field in runtime profiles that
      displays the ClientRequestState execution state (which includes
      the RETRYING and RETRIED states)
    * The Impala Web UI will list all retried queries as being in the
      "RETRIED" state
* Retried queries skip all fe/ planning, authorization, etc.
* This feature is configurable ('retry_failed_queries') and is off by
  default

Refactoring:
* Changes the ClientRequestState so that it can take in an existing
  TExecRequest
    * This is required when retrying queries because the
      TExecRequest of the failed query is copied and used for the
      ClientRequestState of the retried query
* ClientRequestState::ExecState is extended with three new states:
  RETRYING, RETRIED, and UNKNOWN.

Testing:
* Added integration tests in test_query_retries.py, these tests
  consistently pass when run locally
* Ran exhaustive tests

TODO:
* There are some failed tests I am working through
* Additional re-factoring / code cleanup
* Lots more documentation

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
M be/src/service/client-request-state-map.h
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
A be/src/service/retry-work.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
A tests/custom_cluster/test_query_retries.py
16 files changed, 811 insertions(+), 167 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/3
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 3
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 1:

(6 comments)

Pretty major update. Think the feature is coming along and all the core functionality is present. @Thomas addressed your comments and rebased on top of master. As discussed, removed all the changes to status.h and status.cc. Retries are now driven by cluster membership changes or blacklist events.

Still a lot TODO, but appreciate any early feedback. Some additional re-factoring to make the code cleaner is probably necessary, along with a lot more documentation. A lot more testing is necessary as well.

http://gerrit.cloudera.org:8080/#/c/14824/1//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/14824/1//COMMIT_MSG@44
PS1, Line 44: Retries are transparent to the user
> | So, if a query fails and is retried the failed version will just 
Ended up implementing your suggestion for adding a mapping between query ids. See ImpalaServer::client_query_id_mapping_


http://gerrit.cloudera.org:8080/#/c/14824/1/be/src/common/status.h
File be/src/common/status.h:

http://gerrit.cloudera.org:8080/#/c/14824/1/be/src/common/status.h@308
PS1, Line 308: overriding any existing values
> Yeah, legitimate concern. Although FWIW I think we have the same issue with
Removed all of this.


http://gerrit.cloudera.org:8080/#/c/14824/1/be/src/common/status.cc
File be/src/common/status.cc:

http://gerrit.cloudera.org:8080/#/c/14824/1/be/src/common/status.cc@255
PS1, Line 255:     msg_->AddDetail(status.msg().msg());
> I think you're missing a 'msg_->SetStatusProperties(properties)' here
Removed all this.


http://gerrit.cloudera.org:8080/#/c/14824/2/be/src/service/impala-server.h
File be/src/service/impala-server.h:

http://gerrit.cloudera.org:8080/#/c/14824/2/be/src/service/impala-server.h@1007
PS2, Line 1007: 
> line too long (91 > 90)
Done


http://gerrit.cloudera.org:8080/#/c/14824/1/common/thrift/ImpalaService.thrift
File common/thrift/ImpalaService.thrift:

http://gerrit.cloudera.org:8080/#/c/14824/1/common/thrift/ImpalaService.thrift@507
PS1, Line 507: due to cluster membership changes
> I assume ultimately we'll want to use this same flag to control all situati
Done


http://gerrit.cloudera.org:8080/#/c/14824/1/common/thrift/Status.thrift
File common/thrift/Status.thrift:

http://gerrit.cloudera.org:8080/#/c/14824/1/common/thrift/Status.thrift@45
PS1, Line 45:   3: optional TStatusProperties status_properties
> Yeah, we could include it in StatusAuxInfo (although now that StatusAuxInfo
Removed all this.



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 1
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 24 Jan 2020 16:58:21 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 17:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/14824/17/be/src/service/impala-server.h
File be/src/service/impala-server.h:

http://gerrit.cloudera.org:8080/#/c/14824/17/be/src/service/impala-server.h@249
PS17, Line 249:   virtual void Cancel(impala::TStatus& status, const beeswax::QueryHandle& beeswax_handle);
> line too long (91 > 90)
Done



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 17
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Wed, 06 May 2020 01:25:59 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has submitted this change and it was merged. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes (IMPALA-9124).

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the removed node, queries are retried), or (2) if a
query fails and as a result, blacklists a node. Either event is
considered a cluster membership change as it affects what nodes a query
will be scheduled on. The assumption is that a retry of the query with
the updated cluster membership will succeed.

A query retry is modelled as a brand new query, with its own query id.
This simplifies the implementation and the resulting runtime profiles
when queries are retried.

Core Features:
* Retries are transparent to the user; no modification to client
  libraries are necessary to support query retries
* Retried queries skip all fe/ parsing, planning, authorization, etc.
* Retries are configurable ('retry_failed_queries') and are off by
  default

Implementation:
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A new layer of abstraction between the ImpalaServer and
  ClientRequestState has been added; it is called the QueryDriver
* Each ClientRequestState is treated as a single attempt of a query, and
  the QueryDriver owns all ClientRequestStates for a query
* ClientRequestState has a new state object called RetryState; a
  ClientRequestState can either be NOT_RETRIED, RETRYING, or RETRIED
* The QueryDriver owns the TExecRequest for the query as well, it is
  re-used for each query retry
* QueryDrivers and ClientRequestStates are now referenced using a
  QueryHandle

Observability:
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
* Runtime profiles of queries that fail and then are retried will have:
    * "Retry Status: RETRIED"
    * "Retry Cause: [the error that triggered the retry]"
    * "Retried Query Id: [the query id of the retried query]"
* Runtime profiles of the retried query (e.g. the second attempt of the
  query) will include:
    * "Original Query Id: [the query id of the original query]"
* The Impala Web UI will list all retried queries as being in the
  "RETRIED" state

Testing:
* Added E2E tests in test_query_retries.py; looped tests for a few days
* Added a stress test query_retries_stress_runner.py that runs concurrent
  streams of a TPC-{H,DS} workload and randomly kills impalads
* Ran the stress test with various configurations: tpch on parquet,
  tpcds on parquet, tpch 30 GB on parquet (one stream), tpcds 30 GB on
  parquet (one stream), tpch on text, tpcds on text
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures
* Ran 30 GB TPC-DS workload on a 3 node cluster, randomly restarted
  impalads, and manually verified that queries were retried
* Manually tested retries work with various clients, specifically the
  impala-shell and Hue
* Ran core tests and query retry stress test against an ASAN build
* Ran concurrent_select.py to stress query cancellation
* Ran be/ tests against a TSAN build

Limitations:
* There are several limitations that are listed out in the parent JIRA

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Reviewed-on: http://gerrit.cloudera.org:8080/14824
Tested-by: Impala Public Jenkins <im...@cloudera.com>
Reviewed-by: Sahil Takiar <st...@cloudera.com>
---
M be/src/benchmarks/process-wide-locks-benchmark.cc
M be/src/runtime/CMakeLists.txt
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
A be/src/runtime/query-driver.cc
A be/src/runtime/query-driver.h
M be/src/service/CMakeLists.txt
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/control-service.cc
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
R be/src/service/query-driver-map.cc
A be/src/service/query-driver-map.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/testutil/impalad-query-executor.cc
M be/src/testutil/impalad-query-executor.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M tests/common/impala_cluster.py
M tests/common/impala_service.py
A tests/custom_cluster/test_query_retries.py
A tests/stress/query_retries_stress_runner.py
27 files changed, 2,700 insertions(+), 696 deletions(-)

Approvals:
  Impala Public Jenkins: Verified
  Sahil Takiar: Looks good to me, approved

-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: merged
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 33
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 16:

(4 comments)

http://gerrit.cloudera.org:8080/#/c/14824/16/be/src/runtime/coordinator.h
File be/src/runtime/coordinator.h:

http://gerrit.cloudera.org:8080/#/c/14824/16/be/src/runtime/coordinator.h@248
PS16, Line 248: A shared_ptr is used because the
              :   /// Coordinator can outlive its associated QueryDriver.
> This comment needs updating
Done


http://gerrit.cloudera.org:8080/#/c/14824/16/be/src/service/client-request-state.h
File be/src/service/client-request-state.h:

http://gerrit.cloudera.org:8080/#/c/14824/16/be/src/service/client-request-state.h@607
PS16, Line 607: A shared_ptr is used because the
              :   /// ClientRequestState can outlive its associated QueryDriver.
> Update this comment (or remove)
Done


http://gerrit.cloudera.org:8080/#/c/14824/16/be/src/service/impala-beeswax-server.cc
File be/src/service/impala-beeswax-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/16/be/src/service/impala-beeswax-server.cc@212
PS16, Line 212: query_handle;
> Nit: It would be nice to standardize the naming for variables/arguments for
Done


http://gerrit.cloudera.org:8080/#/c/14824/16/be/src/service/impala-server.h
File be/src/service/impala-server.h:

http://gerrit.cloudera.org:8080/#/c/14824/16/be/src/service/impala-server.h@933
PS16, Line 933:   void TUniqueIdToQueryHandle(const TUniqueId& query_id, beeswax::QueryHandle* handle);
              :   void QueryHandleToTUniqueId(const beeswax::QueryHandle& handle, TUniqueId* query_id);
> Nit: Should we rename this to emphasize that this is a beeswax::QueryHandle
Done



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 16
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Wed, 06 May 2020 01:22:46 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 31:

Build started: https://jenkins.impala.io/job/gerrit-verify-dryrun/5839/ DRY_RUN=true


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 31
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 15 May 2020 03:09:39 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 8:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/14824/8/tests/stress/query_retries_stress_runner.py
File tests/stress/query_retries_stress_runner.py:

http://gerrit.cloudera.org:8080/#/c/14824/8/tests/stress/query_retries_stress_runner.py@160
PS8, Line 160: e
> flake8: E722 do not use bare except'
Done


http://gerrit.cloudera.org:8080/#/c/14824/8/tests/stress/query_retries_stress_runner.py@164
PS8, Line 164: e
> flake8: E722 do not use bare except'
Done



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 8
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Tue, 31 Mar 2020 18:08:24 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9124 [POC][WIP]: Transparently retry queries that fail due to cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9124 [POC][WIP]: Transparently retry queries that fail due to cluster membership changes
......................................................................


Patch Set 1:

Build Failed 

https://jenkins.impala.io/job/gerrit-code-review-checks/5245/ : Initial code review checks failed. See linked job for details on the failure.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 1
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Michael Ho <mi...@gmail.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Wed, 11 Dec 2019 01:53:45 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 27:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/6041/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 27
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Tue, 12 May 2020 17:52:33 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 5:

(1 comment)

* Fixing clang-tidy errors
* Fixed a bug where timed-out queries were being retried
* Actually honoring the setting of 'retry_failed_queries'

http://gerrit.cloudera.org:8080/#/c/14824/4/be/src/service/impala-server.h
File be/src/service/impala-server.h:

http://gerrit.cloudera.org:8080/#/c/14824/4/be/src/service/impala-server.h@1007
PS4, Line 1007:   /// order to avoid query compilation and planning again. Once the new query is
> line too long (91 > 90)
Done



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 5
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 24 Jan 2020 22:34:34 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 21:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/5973/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 21
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Wed, 06 May 2020 03:56:38 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Joe McDonnell, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#10).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes (IMPALA-9124).

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the removed node, queries are retried), or (2) if a
query fails and as a result, blacklists a node. Either event is
considered a cluster membership change as it affects what nodes a query
will be scheduled on. The assumption is that a retry of the query with
the updated cluster membership will succeed.

A query retry is modelled as a brand new query, with its own query id.
This simplifies the implementation and the resulting runtime profiles
when queries are retried.

Core Features:
* Retries are transparent to the user; no modification to client
  libraries are necessary to support query retries
* Retried queries skip all fe/ parsing, planning, authorization, etc.
* Retries are configurable ('retry_failed_queries') and are off by
  default

Implementation:
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A new layer of abstraction between the ImpalaServer and
  ClientRequestState has been added; it is called the QueryDriver
* Each ClientRequestState is treated as a single attempt of a query, and
  the QueryDriver owns all ClientRequestStates for a query
* ClientRequestState has a new state object called RetryState; a
  ClientRequestState can either be NOT_RETRIED, RETRYING, or RETRIED
* The QueryDriver owns the TExecRequest for the query as well, it is
  re-used for each query retry

Observability:
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
* Runtime profiles of queries that fail and then are retried will have:
    * "Retry Status: RETRIED"
    * "Retry Cause: [the error that triggered the retry]"
    * "Retried Query Id: [the query id of the retried query]"
* Runtime profiles of the retried query (e.g. the second attempt of the
  query) will include:
    * "Original Query Id: [the query id of the original query]"
* The Impala Web UI will list all retried queries as being in the
  "RETRIED" state

Testing:
* Added E2E tests in test_query_retries.py
* Added a stress test query_retries_stress_runner.py that runs concurrent
  streams of a TPC workload and randomly kills impalads
* Ran the stress test with various configurations: tpch on parquet,
  tpcds on parquet, tpch 30 GB on parquet (one stream), tpcds 30 GB on
  parquet (one stream), tpch on text, tpcds on text
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures
* Ran 30 GB TPC-DS workload on a 3 node cluster, randomly restarted
  impalads, and manually verified that queries were retried
* Manually tested retries work with various clients, specifically the
  impala-shell and Hue

Limitations:
* IMPALA-9225: A query cannot be retried once any results from the original
  query have been fetched
* IMPALA-9200: A query can currently only be retried once

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/runtime/CMakeLists.txt
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
A be/src/runtime/query-driver.cc
A be/src/runtime/query-driver.h
M be/src/service/CMakeLists.txt
D be/src/service/client-request-state-map.h
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/control-service.cc
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
R be/src/service/query-driver-map.cc
A be/src/service/query-driver-map.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M common/thrift/generate_error_codes.py
M tests/common/impala_cluster.py
M tests/common/impala_service.py
A tests/custom_cluster/test_query_retries.py
A tests/stress/query_retries_stress_runner.py
26 files changed, 2,253 insertions(+), 448 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/10
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 10
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 11:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/14824/11/be/src/runtime/query-driver.cc
File be/src/runtime/query-driver.cc:

http://gerrit.cloudera.org:8080/#/c/14824/11/be/src/runtime/query-driver.cc@70
PS11, Line 70: shared_ptr<ClientRequestState> QueryDriver::GetClientRequestState(const TUniqueId& query_id) {
line too long (94 > 90)



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 11
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Thu, 30 Apr 2020 00:04:39 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 5:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/5517/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 5
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 24 Jan 2020 23:18:52 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 10:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/5914/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 10
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Thu, 30 Apr 2020 00:47:53 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Joe McDonnell (Code Review)" <ge...@cloudera.org>.
Joe McDonnell has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 21: Code-Review+1

(2 comments)

This is looking good to me.

http://gerrit.cloudera.org:8080/#/c/14824/21/be/src/runtime/coordinator.cc
File be/src/runtime/coordinator.cc:

http://gerrit.cloudera.org:8080/#/c/14824/21/be/src/runtime/coordinator.cc@1071
PS21, Line 1071:   transform(failed_backend_states.begin(), failed_backend_states.end(),
               :       back_inserter(backend_addresses), [](BackendState* backend_state) -> string {
               :         return TNetworkAddressToString(backend_state->krpc_impalad_address());
               :       });
Purely style nit: I think this would be cleaner as a for-loop.

for (BackendState* backend_state : failed_backend_states) {
  backend_addresses.push_back(TNetworkAddressToString(...));
}


http://gerrit.cloudera.org:8080/#/c/14824/21/be/src/runtime/coordinator.cc@1077
PS21, Line 1077:   for_each(failed_backend_states.begin(), failed_backend_states.end(),
               :       [&retryable_status](BackendState* backend_state) {
               :         retryable_status.MergeStatus(
               :             FromKuduStatus(backend_state->exec_rpc_status(), "Exec() rpc failed"));
               :       });
Purely style nit: I think this would be cleaner as a for-loop:

for (BackendState* backend_state : failed_backend_states) {
  retryable_status.MergeStatus(FromKuduStatus(...), ...);
}



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 21
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Wed, 06 May 2020 23:14:56 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Joe McDonnell (Code Review)" <ge...@cloudera.org>.
Joe McDonnell has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 16:

(4 comments)

Took another pass. It is mostly nits about beeswax::QueryHandle vs impala::QueryHandle and some stale comments.

http://gerrit.cloudera.org:8080/#/c/14824/16/be/src/runtime/coordinator.h
File be/src/runtime/coordinator.h:

http://gerrit.cloudera.org:8080/#/c/14824/16/be/src/runtime/coordinator.h@248
PS16, Line 248: A shared_ptr is used because the
              :   /// Coordinator can outlive its associated QueryDriver.
This comment needs updating


http://gerrit.cloudera.org:8080/#/c/14824/16/be/src/service/client-request-state.h
File be/src/service/client-request-state.h:

http://gerrit.cloudera.org:8080/#/c/14824/16/be/src/service/client-request-state.h@607
PS16, Line 607: A shared_ptr is used because the
              :   /// ClientRequestState can outlive its associated QueryDriver.
Update this comment (or remove)


http://gerrit.cloudera.org:8080/#/c/14824/16/be/src/service/impala-beeswax-server.cc
File be/src/service/impala-beeswax-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/16/be/src/service/impala-beeswax-server.cc@212
PS16, Line 212: query_handle;
Nit: It would be nice to standardize the naming for variables/arguments for beeswax::QueryHandle vs impala::QueryHandle across the functions in this file. e.g. the beeswax::QueryHandle is always 'handle' or 'beeswax_handle' or whatever and the impala::QueryHandle is always 'query_handle' or 'impala_query_handle' or whatever. I think I would lean toward beeswax::QueryHandle -> 'beeswax_handle' and impala::QueryHandle -> 'query_handle', but I'm flexible.

This variability predates your change, but I don't think there is anything stopping us from standardizing.


http://gerrit.cloudera.org:8080/#/c/14824/16/be/src/service/impala-server.h
File be/src/service/impala-server.h:

http://gerrit.cloudera.org:8080/#/c/14824/16/be/src/service/impala-server.h@933
PS16, Line 933:   void TUniqueIdToQueryHandle(const TUniqueId& query_id, beeswax::QueryHandle* handle);
              :   void QueryHandleToTUniqueId(const beeswax::QueryHandle& handle, TUniqueId* query_id);
Nit: Should we rename this to emphasize that this is a beeswax::QueryHandle? Something like:
TUniqueIdToBeeswaxHandle()
BeeswaxHandleToTUniqueId()



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 16
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Tue, 05 May 2020 23:38:53 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 32: Verified+1


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 32
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 15 May 2020 20:06:19 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 11:

(10 comments)

Patch set 10 is a rebase. Patch set 11 address the following comments. Addressed / responded to the rest of the comments from Thomas. Addressed most (but not all) comments from Joe.

http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/runtime/query-driver.h
File be/src/runtime/query-driver.h:

http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/runtime/query-driver.h@113
PS9, Line 113: (2) the
             :   /// query has not already been ret
> not sure what this is supposed to mean
revised


http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/runtime/query-driver.h@116
PS9, Line 116: d if
             :   /// there has been a c
> I think this should be singular "a statestore update"
Done


http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/runtime/query-driver.h@122
PS9, Line 122:   /// is set to true if the query was actually retried, false otherwise. This method is
             :   /// idempotent, it can safely be called multiple tim
> From reading through TryQueryRetry(), it looks like this is safe to call mu
updated docs to mention that method is idempotent


http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/service/client-request-state.h
File be/src/service/client-request-state.h:

http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/service/client-request-state.h@84
PS9, Line 84: ;
> I don't think this is used anywhere?
Done


http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/service/impala-hs2-server.cc
File be/src/service/impala-hs2-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/service/impala-hs2-server.cc@148
PS9, Line 148:   shared_ptr<QueryDriver> query_driver;
> I don't think this is used anywhere?
Done


http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/service/impala-hs2-server.cc@763
PS9, Line 763:   HS2_RETURN_IF_ERROR(
> I had the same concern.
yeah, thats a good point, thanks for catching that. changed it to use a shared_ptr


http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/service/impala-hs2-server.cc@1039
PS9, Line 1039:   // The original query profile should still be accessible via the web ui. Don't use
> I think its definitely necessary that we provide a way for all relevant pro
agree this is an issue. the reason i went with the current approach is so that all methods in the hs2/beeswax interface follow the same "transparent" pattern.
i think it might be slightly more intuitive to return the profile of the successful query rather than the originally failed one. although I think returning the original, failed profile could make sense as well. I don't know if one approach is necessarily that much better than the other.
i agree that there should be a way to fetch the failed profiles via the hs2 interface, I'm planning to tackle that in IMPALA-9229


http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/service/impala-server.h
File be/src/service/impala-server.h:

http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/service/impala-server.h@673
PS9, Line 673: std::shared_ptr<Clie
> Stylistically, I personally don't like mutable references for out arguments
Done since I had to change this all the shared_ptr, so now it is "shared_ptr<ClientRequestState>*"


http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/service/impala-server.cc
File be/src/service/impala-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/9/be/src/service/impala-server.cc@1453
PS9, Line 1453:   // ClientRequestState::output_exprs_, which
> Nit: you could move this declaration down past the first BlockOnWait()
Done


http://gerrit.cloudera.org:8080/#/c/14824/9/common/thrift/ImpalaService.thrift
File common/thrift/ImpalaService.thrift:

http://gerrit.cloudera.org:8080/#/c/14824/9/common/thrift/ImpalaService.thrift@523
PS9, Line 523: 
> typo
Done



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 11
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Thu, 30 Apr 2020 00:04:51 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 2:

Build Failed 

https://jenkins.impala.io/job/gerrit-code-review-checks/5514/ : Initial code review checks failed. See linked job for details on the failure.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 2
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 24 Jan 2020 17:35:22 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Joe McDonnell, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#13).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes (IMPALA-9124).

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the removed node, queries are retried), or (2) if a
query fails and as a result, blacklists a node. Either event is
considered a cluster membership change as it affects what nodes a query
will be scheduled on. The assumption is that a retry of the query with
the updated cluster membership will succeed.

A query retry is modelled as a brand new query, with its own query id.
This simplifies the implementation and the resulting runtime profiles
when queries are retried.

Core Features:
* Retries are transparent to the user; no modification to client
  libraries are necessary to support query retries
* Retried queries skip all fe/ parsing, planning, authorization, etc.
* Retries are configurable ('retry_failed_queries') and are off by
  default

Implementation:
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A new layer of abstraction between the ImpalaServer and
  ClientRequestState has been added; it is called the QueryDriver
* Each ClientRequestState is treated as a single attempt of a query, and
  the QueryDriver owns all ClientRequestStates for a query
* ClientRequestState has a new state object called RetryState; a
  ClientRequestState can either be NOT_RETRIED, RETRYING, or RETRIED
* The QueryDriver owns the TExecRequest for the query as well, it is
  re-used for each query retry

Observability:
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
* Runtime profiles of queries that fail and then are retried will have:
    * "Retry Status: RETRIED"
    * "Retry Cause: [the error that triggered the retry]"
    * "Retried Query Id: [the query id of the retried query]"
* Runtime profiles of the retried query (e.g. the second attempt of the
  query) will include:
    * "Original Query Id: [the query id of the original query]"
* The Impala Web UI will list all retried queries as being in the
  "RETRIED" state

Testing:
* Added E2E tests in test_query_retries.py
* Added a stress test query_retries_stress_runner.py that runs concurrent
  streams of a TPC workload and randomly kills impalads
* Ran the stress test with various configurations: tpch on parquet,
  tpcds on parquet, tpch 30 GB on parquet (one stream), tpcds 30 GB on
  parquet (one stream), tpch on text, tpcds on text
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures
* Ran 30 GB TPC-DS workload on a 3 node cluster, randomly restarted
  impalads, and manually verified that queries were retried
* Manually tested retries work with various clients, specifically the
  impala-shell and Hue

Limitations:
* IMPALA-9225: A query cannot be retried once any results from the original
  query have been fetched
* IMPALA-9200: A query can currently only be retried once

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/runtime/CMakeLists.txt
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
A be/src/runtime/query-driver.cc
A be/src/runtime/query-driver.h
M be/src/service/CMakeLists.txt
D be/src/service/client-request-state-map.h
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/control-service.cc
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
R be/src/service/query-driver-map.cc
A be/src/service/query-driver-map.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M common/thrift/generate_error_codes.py
M tests/common/impala_cluster.py
M tests/common/impala_service.py
A tests/custom_cluster/test_query_retries.py
A tests/stress/query_retries_stress_runner.py
26 files changed, 2,267 insertions(+), 427 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/13
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 13
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Joe McDonnell, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#19).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes (IMPALA-9124).

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the removed node, queries are retried), or (2) if a
query fails and as a result, blacklists a node. Either event is
considered a cluster membership change as it affects what nodes a query
will be scheduled on. The assumption is that a retry of the query with
the updated cluster membership will succeed.

A query retry is modelled as a brand new query, with its own query id.
This simplifies the implementation and the resulting runtime profiles
when queries are retried.

Core Features:
* Retries are transparent to the user; no modification to client
  libraries are necessary to support query retries
* Retried queries skip all fe/ parsing, planning, authorization, etc.
* Retries are configurable ('retry_failed_queries') and are off by
  default

Implementation:
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A new layer of abstraction between the ImpalaServer and
  ClientRequestState has been added; it is called the QueryDriver
* Each ClientRequestState is treated as a single attempt of a query, and
  the QueryDriver owns all ClientRequestStates for a query
* ClientRequestState has a new state object called RetryState; a
  ClientRequestState can either be NOT_RETRIED, RETRYING, or RETRIED
* The QueryDriver owns the TExecRequest for the query as well, it is
  re-used for each query retry

Observability:
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
* Runtime profiles of queries that fail and then are retried will have:
    * "Retry Status: RETRIED"
    * "Retry Cause: [the error that triggered the retry]"
    * "Retried Query Id: [the query id of the retried query]"
* Runtime profiles of the retried query (e.g. the second attempt of the
  query) will include:
    * "Original Query Id: [the query id of the original query]"
* The Impala Web UI will list all retried queries as being in the
  "RETRIED" state

Testing:
* Added E2E tests in test_query_retries.py; looped tests for a few days
* Added a stress test query_retries_stress_runner.py that runs concurrent
  streams of a TPC workload and randomly kills impalads
* Ran the stress test with various configurations: tpch on parquet,
  tpcds on parquet, tpch 30 GB on parquet (one stream), tpcds 30 GB on
  parquet (one stream), tpch on text, tpcds on text
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures
* Ran 30 GB TPC-DS workload on a 3 node cluster, randomly restarted
  impalads, and manually verified that queries were retried
* Manually tested retries work with various clients, specifically the
  impala-shell and Hue
* Ran core tests and query retry stress test against an ASAN build
* Ran concurrent_select.py to stress query cancellation

Limitations:
* There are several limitations that are listed out in the parent JIRA

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/benchmarks/process-wide-locks-benchmark.cc
M be/src/runtime/CMakeLists.txt
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
A be/src/runtime/query-driver.cc
A be/src/runtime/query-driver.h
M be/src/service/CMakeLists.txt
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/control-service.cc
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
R be/src/service/query-driver-map.cc
A be/src/service/query-driver-map.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/testutil/impalad-query-executor.cc
M be/src/testutil/impalad-query-executor.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M common/thrift/generate_error_codes.py
M tests/common/impala_cluster.py
M tests/common/impala_service.py
A tests/custom_cluster/test_query_retries.py
A tests/stress/query_retries_stress_runner.py
28 files changed, 2,444 insertions(+), 510 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/19
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 19
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Joe McDonnell, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#17).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes (IMPALA-9124).

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the removed node, queries are retried), or (2) if a
query fails and as a result, blacklists a node. Either event is
considered a cluster membership change as it affects what nodes a query
will be scheduled on. The assumption is that a retry of the query with
the updated cluster membership will succeed.

A query retry is modelled as a brand new query, with its own query id.
This simplifies the implementation and the resulting runtime profiles
when queries are retried.

Core Features:
* Retries are transparent to the user; no modification to client
  libraries are necessary to support query retries
* Retried queries skip all fe/ parsing, planning, authorization, etc.
* Retries are configurable ('retry_failed_queries') and are off by
  default

Implementation:
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A new layer of abstraction between the ImpalaServer and
  ClientRequestState has been added; it is called the QueryDriver
* Each ClientRequestState is treated as a single attempt of a query, and
  the QueryDriver owns all ClientRequestStates for a query
* ClientRequestState has a new state object called RetryState; a
  ClientRequestState can either be NOT_RETRIED, RETRYING, or RETRIED
* The QueryDriver owns the TExecRequest for the query as well, it is
  re-used for each query retry

Observability:
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
* Runtime profiles of queries that fail and then are retried will have:
    * "Retry Status: RETRIED"
    * "Retry Cause: [the error that triggered the retry]"
    * "Retried Query Id: [the query id of the retried query]"
* Runtime profiles of the retried query (e.g. the second attempt of the
  query) will include:
    * "Original Query Id: [the query id of the original query]"
* The Impala Web UI will list all retried queries as being in the
  "RETRIED" state

Testing:
* Added E2E tests in test_query_retries.py; looped tests for a few days
* Added a stress test query_retries_stress_runner.py that runs concurrent
  streams of a TPC workload and randomly kills impalads
* Ran the stress test with various configurations: tpch on parquet,
  tpcds on parquet, tpch 30 GB on parquet (one stream), tpcds 30 GB on
  parquet (one stream), tpch on text, tpcds on text
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures
* Ran 30 GB TPC-DS workload on a 3 node cluster, randomly restarted
  impalads, and manually verified that queries were retried
* Manually tested retries work with various clients, specifically the
  impala-shell and Hue
* Ran core tests and query retry stress test against an ASAN build
* Ran concurrent_select.py to stress query cancellation

Limitations:
* There are several limitations that are listed out in the parent JIRA

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/runtime/CMakeLists.txt
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
A be/src/runtime/query-driver.cc
A be/src/runtime/query-driver.h
M be/src/service/CMakeLists.txt
D be/src/service/client-request-state-map.h
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/control-service.cc
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
R be/src/service/query-driver-map.cc
A be/src/service/query-driver-map.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/testutil/impalad-query-executor.cc
M be/src/testutil/impalad-query-executor.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M common/thrift/generate_error_codes.py
M tests/common/impala_cluster.py
M tests/common/impala_service.py
A tests/custom_cluster/test_query_retries.py
A tests/stress/query_retries_stress_runner.py
28 files changed, 2,370 insertions(+), 514 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/17
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 17
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 30: Code-Review+2

Re-based, carrying +2. I'm going to run the pre-commit job to make sure it passes, but I'm also going run the stress tests again locally to make sure they still pass. I'm also going to do one final read-through of the code to make sure we didn't miss anything.


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 30
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Thu, 14 May 2020 20:56:29 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 13:

(5 comments)

addressed comments, fixed a possible race condition when a query gets cancelled and unregistered while a retry is being setup.

http://gerrit.cloudera.org:8080/#/c/14824/13/be/src/runtime/query-driver.h
File be/src/runtime/query-driver.h:

http://gerrit.cloudera.org:8080/#/c/14824/13/be/src/runtime/query-driver.h@169
PS13, Line 169: 
> ClientRequestState has a pointer back to the QueryDriver (as does the Coord
Fixed. The Coordinator and ClientRequestState now take a shared_ptr to the QueryDriver


http://gerrit.cloudera.org:8080/#/c/14824/13/be/src/runtime/query-driver.cc
File be/src/runtime/query-driver.cc:

http://gerrit.cloudera.org:8080/#/c/14824/13/be/src/runtime/query-driver.cc@161
PS13, Line 161: void QueryDriver::RetryQueryFromThread(const Status& error) {
> Is there anything that prevents the QueryDriver from being deleted while th
hmm yeah that's true. I changed it so that RetryQueryFromThread takes in a shared_ptr to the current QueryDriver. its a bit odd that the method has to take in a reference to itself, but I couldn't see a better way.


http://gerrit.cloudera.org:8080/#/c/14824/13/be/src/runtime/query-driver.cc@178
PS13, Line 178:   // Cancel the query.
> Is this necessary? At least in the case of Coordinator calling TryQueryRetr
I don't think 'ImpalaServer::CancelFromThreadPool' (which calls TryQueryRetry() will call Cancel. Its probably possible to remove this, but I thought it was a nice property that you want to ensure the query is cancelled before trying to retry it.


http://gerrit.cloudera.org:8080/#/c/14824/13/be/src/runtime/query-driver.cc@179
PS13, Line 179: true, nullptr
> Both of these parameters seem wrong - you note above that a query could get
I changed the 'check_inflight' parameter to false.
ClientRequestState::MarkAsRetrying explicitly sets the query_status_ in the ClientRequestState. The issue with calling Cancel with a cause is that it will call UpdateQueryStatus, which sets the ExecState to ERROR, which you don't want because the ERROR state will get exposed to the client, which you don't want to happen since the query is being retried.


http://gerrit.cloudera.org:8080/#/c/14824/13/be/src/runtime/query-driver.cc@281
PS13, Line 281:   // TODO: IMPALA-9502: Avoid copying TExecRequest when retrying queries
> Out of curiosity, what are the issues here?
hmm can't remember, I replaced this with 'move(exec_request_)' and all the tests and a run of the stress test passes. I added this a long time ago on a much older version of this feature, so it is possible it got fixed as some point inadvertently.



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 13
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 01 May 2020 21:33:54 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 25:

(4 comments)

http://gerrit.cloudera.org:8080/#/c/14824/25/be/src/service/impala-server.cc
File be/src/service/impala-server.cc:

http://gerrit.cloudera.org:8080/#/c/14824/25/be/src/service/impala-server.cc@642
PS25, Line 642:     Status status = GetQueryHandle(query_id, &query_handle, /*return_unregistered=*/ true);
line too long (91 > 90)


http://gerrit.cloudera.org:8080/#/c/14824/25/be/src/service/impala-server.cc@644
PS25, Line 644:       ClientRequestState* request_state = query_handle.request_state; 
line has trailing whitespace


http://gerrit.cloudera.org:8080/#/c/14824/25/be/src/service/impala-server.cc@710
PS25, Line 710:     Status status = GetQueryHandle(query_id, &query_handle, /*return_unregistered=*/ true);
line too long (91 > 90)


http://gerrit.cloudera.org:8080/#/c/14824/25/be/src/service/impala-server.cc@712
PS25, Line 712:       ClientRequestState* request_state = query_handle.request_state; 
line has trailing whitespace



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 25
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 08 May 2020 19:14:33 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Joe McDonnell (Code Review)" <ge...@cloudera.org>.
Joe McDonnell has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 13:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/14824/13/be/src/runtime/query-driver.h
File be/src/runtime/query-driver.h:

http://gerrit.cloudera.org:8080/#/c/14824/13/be/src/runtime/query-driver.h@169
PS13, Line 169: 
> Fixed. The Coordinator and ClientRequestState now take a shared_ptr to the 
I should have mentioned that the shared_ptr approach also requires special teardown. We need to break the cycle of shared_ptrs, so there isn't a resource leak.

If A holds a shared_ptr to B and B holds a shared_ptr to A, they will both continue to exist even if the last external shared_ptr is destructed.



-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 13
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Fri, 01 May 2020 22:52:27 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Hello Thomas Tauber-Marshall, Joe McDonnell, Impala Public Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/14824

to look at the new patch set (#23).

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................

IMPALA-9199: Add support for single query retries on cluster membership changes

Adds the core logic for transparently retrying queries that fail due to
cluster membership changes (IMPALA-9124).

Query retries are triggered if (1) a node has been removed from the
cluster membership by a statestore update (rather than cancelling all
queries running on the removed node, queries are retried), or (2) if a
query fails and as a result, blacklists a node. Either event is
considered a cluster membership change as it affects what nodes a query
will be scheduled on. The assumption is that a retry of the query with
the updated cluster membership will succeed.

A query retry is modelled as a brand new query, with its own query id.
This simplifies the implementation and the resulting runtime profiles
when queries are retried.

Core Features:
* Retries are transparent to the user; no modification to client
  libraries are necessary to support query retries
* Retried queries skip all fe/ parsing, planning, authorization, etc.
* Retries are configurable ('retry_failed_queries') and are off by
  default

Implementation:
* When a query is retried, the original query is cancelled, the new
  query is created, registered, and started, and then the original query
  is closed
* A new layer of abstraction between the ImpalaServer and
  ClientRequestState has been added; it is called the QueryDriver
* Each ClientRequestState is treated as a single attempt of a query, and
  the QueryDriver owns all ClientRequestStates for a query
* ClientRequestState has a new state object called RetryState; a
  ClientRequestState can either be NOT_RETRIED, RETRYING, or RETRIED
* The QueryDriver owns the TExecRequest for the query as well, it is
  re-used for each query retry

Observability:
* Users can tell if a query is retried using runtime profiles and the
  Impala Web UI
* Runtime profiles of queries that fail and then are retried will have:
    * "Retry Status: RETRIED"
    * "Retry Cause: [the error that triggered the retry]"
    * "Retried Query Id: [the query id of the retried query]"
* Runtime profiles of the retried query (e.g. the second attempt of the
  query) will include:
    * "Original Query Id: [the query id of the original query]"
* The Impala Web UI will list all retried queries as being in the
  "RETRIED" state

Testing:
* Added E2E tests in test_query_retries.py; looped tests for a few days
* Added a stress test query_retries_stress_runner.py that runs concurrent
  streams of a TPC workload and randomly kills impalads
* Ran the stress test with various configurations: tpch on parquet,
  tpcds on parquet, tpch 30 GB on parquet (one stream), tpcds 30 GB on
  parquet (one stream), tpch on text, tpcds on text
* Ran exhaustive tests
* Ran exhaustive tests with 'retry_failed_queries' set to true, no
  unexpected failures
* Ran 30 GB TPC-DS workload on a 3 node cluster, randomly restarted
  impalads, and manually verified that queries were retried
* Manually tested retries work with various clients, specifically the
  impala-shell and Hue
* Ran core tests and query retry stress test against an ASAN build
* Ran concurrent_select.py to stress query cancellation
* Ran be/ tests against a TSAN build, filed IMPALA-9730 as a follow up

Limitations:
* There are several limitations that are listed out in the parent JIRA

Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
---
M be/src/benchmarks/process-wide-locks-benchmark.cc
M be/src/runtime/CMakeLists.txt
M be/src/runtime/coordinator.cc
M be/src/runtime/coordinator.h
A be/src/runtime/query-driver.cc
A be/src/runtime/query-driver.h
M be/src/service/CMakeLists.txt
M be/src/service/client-request-state.cc
M be/src/service/client-request-state.h
M be/src/service/control-service.cc
M be/src/service/impala-beeswax-server.cc
M be/src/service/impala-hs2-server.cc
M be/src/service/impala-http-handler.cc
M be/src/service/impala-server.cc
M be/src/service/impala-server.h
R be/src/service/query-driver-map.cc
A be/src/service/query-driver-map.h
M be/src/service/query-options.cc
M be/src/service/query-options.h
M be/src/testutil/impalad-query-executor.cc
M be/src/testutil/impalad-query-executor.h
M common/thrift/ImpalaInternalService.thrift
M common/thrift/ImpalaService.thrift
M common/thrift/generate_error_codes.py
M tests/common/impala_cluster.py
M tests/common/impala_service.py
A tests/custom_cluster/test_query_retries.py
A tests/stress/query_retries_stress_runner.py
28 files changed, 2,458 insertions(+), 510 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/24/14824/23
-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 23
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Joe McDonnell <jo...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>

[Impala-ASF-CR] IMPALA-9199: Add support for single query retries on cluster membership changes

Posted by "Sahil Takiar (Code Review)" <ge...@cloudera.org>.
Sahil Takiar has posted comments on this change. ( http://gerrit.cloudera.org:8080/14824 )

Change subject: IMPALA-9199: Add support for single query retries on cluster membership changes
......................................................................


Patch Set 7:

Instead of creating a separate doc, think it might be easier to just post comments here.

I've done some re-factoring locally and this is what I have so far:

* Created a "QueryDriver", which (as suggested by Thomas) is a layer of abstraction of on top of ClientRequestState
* "QueryDriver" owns the ClientRequestState; all access to the ClientRequestState goes through the QueryDriver
* Since the first patch will only support a single retry, the QueryDriver has a regular client_request_state_ and a retried_client_request_state_ (one for the original query and the other for the retried query)
* QueryDriver owns the TExecRequest that is used by both the original and retried queries
* The "transparent" part of this feature is now handled by the QueryDriver (since the QueryDriver owns all access to the ClientRequestState)
* The QueryDriver has two methods: GetActiveClientRequestState() and GetClientRequestState(query_id)
* GetActiveClientRequestState() returns the CRS for the "active" query - e.g. if the query has not been retried yet, it returns the original query; if the query has been retried, it returns the retried query
* GetClientRequestState(query_id) returns the CRS for the query that matches the given 'query_id' - this allows clients to get the CRS of exactly the query_id they are looking for
* ImpalaServer creates a QueryDriver for each query; the ImpalaServer::client_request_state_map_ has been replaced with the ImpalaServer::query_driver_map_ which maps query ids to QueryDrivers
* The query_driver_map_ can have key, values pairs with the same value (e.g. the same QueryDriver)
* Moved all of the retry code (along with the retry threadpool) into QueryDriver (specifically QueryDriver::RetryQueryFromThreadPool)

Example:

* Client submits a query for execution
* Impala creates a QueryDriver for the query
* QueryDriver::CreateClientRequestState creates a ClientRequestState for the query (this CRS is owned by the QueryDriver)
* ImpalaServer::RegisterQuery "registers" the query and adds an entry to ImpalaServer::query_driver_map_ that maps the query id to the QueryDriver
* QueryDriver::RunFrontendPlanner creates the TExecRequest (owned by the QueryDriver)
* Eventually, the query fails and is retried
* QueryDriver::RetryQueryFromThreadPool contains all the retry logic (moved from ImpalaServer)
* It cancels the original query, creates the new one and runs it
* It calls ImpalaServer::RegisterQuery again, but with the retried query id
* It manipulates some internal pointers so that retried_client_request_state_ now points to the CRS of the retried query


-- 
To view, visit http://gerrit.cloudera.org:8080/14824
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I2e4a0e72a9bf8ec10b91639aefd81bef17886ddd
Gerrit-Change-Number: 14824
Gerrit-PatchSet: 7
Gerrit-Owner: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Sahil Takiar <st...@cloudera.com>
Gerrit-Reviewer: Thomas Tauber-Marshall <tm...@cloudera.com>
Gerrit-Comment-Date: Thu, 13 Feb 2020 19:05:45 +0000
Gerrit-HasComments: No