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 <impala-public-jenkins@cloudera.com>
Reviewed-by: Sahil Takiar <stakiar@cloudera.com>
This commit is contained in:
Sahil Takiar
2019-10-10 15:30:20 -07:00
committed by Sahil Takiar
parent fcf08d1822
commit bd4d01a379
27 changed files with 2700 additions and 696 deletions

View File

@@ -46,7 +46,7 @@
/// This tests the performance of the following process wide locks:
//
/// 1. qs_map_lock_ (Sharded)
/// TODO: client_request_state_map_lock_ (Sharded)
/// TODO: query_driver_map_lock_ (Sharded)
//
/// A reasonable amount of queries are created and accessed multiple times via the
/// QueryExecMgr's APIs to benchmark the time taken to acquire the lock and retrieve

View File

@@ -58,6 +58,7 @@ add_library(Runtime
lib-cache.cc
mem-tracker.cc
mem-pool.cc
query-driver.cc
query-exec-mgr.cc
query-state.cc
test-env.cc

View File

@@ -41,6 +41,7 @@
#include "runtime/fragment-instance-state.h"
#include "runtime/hdfs-fs-cache.h"
#include "runtime/query-exec-mgr.h"
#include "runtime/query-driver.h"
#include "runtime/query-state.h"
#include "scheduling/admission-controller.h"
#include "scheduling/query-schedule.h"
@@ -115,7 +116,8 @@ static const int MAX_BROADCAST_FILTER_PRODUCERS = 3;
Coordinator::Coordinator(ClientRequestState* parent, const QuerySchedule& schedule,
RuntimeProfile::EventSequence* events)
: parent_request_state_(parent),
: parent_query_driver_(parent->parent_driver()),
parent_request_state_(parent),
schedule_(schedule),
filter_mode_(schedule.query_options().runtime_filter_mode),
obj_pool_(new ObjectPool()),
@@ -492,10 +494,13 @@ Status Coordinator::StartBackendExec() {
// One of the backends failed to startup, so we cancel the other ones.
CancelBackends(/*fire_and_forget=*/ true);
WaitOnExecRpcs();
vector<BackendState*> failed_backend_states;
for (BackendState* backend_state : backend_states_) {
// If Exec() rpc failed for a reason besides being aborted, blacklist the executor.
// If Exec() rpc failed for a reason besides being aborted, blacklist the executor
// and retry the query.
if (!backend_state->exec_rpc_status().ok()
&& !backend_state->exec_rpc_status().IsAborted()) {
failed_backend_states.push_back(backend_state);
LOG(INFO) << "Blacklisting " << backend_state->impalad_address()
<< " because an Exec() rpc to it failed.";
const BackendDescriptorPB& be_desc = backend_state->exec_params()->be_desc;
@@ -503,6 +508,9 @@ Status Coordinator::StartBackendExec() {
FromKuduStatus(backend_state->exec_rpc_status(), "Exec() rpc failed"));
}
}
if (!failed_backend_states.empty()) {
HandleFailedExecRpcs(failed_backend_states);
}
VLOG_QUERY << "query startup cancelled due to a failed Exec() rpc: "
<< exec_rpc_status;
return UpdateExecState(exec_rpc_status, nullptr, FLAGS_hostname);
@@ -894,6 +902,9 @@ Status Coordinator::UpdateBackendExecStatus(const ReportExecStatusRequestPB& req
if (thrift_profiles.__isset.host_profile) {
backend_state->UpdateHostProfile(thrift_profiles.host_profile);
}
// Set by ApplyExecStatusReport, contains all the AuxErrorInfoPB objects in
// ReportExecStatusRequestPB.
vector<AuxErrorInfoPB> aux_error_info;
if (backend_state->ApplyExecStatusReport(request, thrift_profiles, &exec_summary_,
@@ -913,6 +924,21 @@ Status Coordinator::UpdateBackendExecStatus(const ReportExecStatusRequestPB& req
bool is_fragment_failure;
TUniqueId failed_instance_id;
Status status = backend_state->GetStatus(&is_fragment_failure, &failed_instance_id);
// Iterate through all AuxErrorInfoPB objects, and use each one to possibly blacklist
// any "faulty" nodes.
Status retryable_status = UpdateBlacklistWithAuxErrorInfo(
&aux_error_info, status, backend_state);
// If any nodes were blacklisted, retry the query. This needs to be done before
// UpdateExecState is called with the error status to avoid exposing the error to any
// clients. If a retry is attempted, the ClientRequestState::query_status_ will be
// set by TryQueryRetry, which prevents the error status from being exposed to any
// clients.
if (!retryable_status.ok()) {
parent_query_driver_->TryQueryRetry(parent_request_state_, &retryable_status);
}
if (!status.ok()) {
// We may start receiving status reports before all exec rpcs are complete.
// Can't apply state transition until no more exec rpcs will be sent.
@@ -923,9 +949,14 @@ Status Coordinator::UpdateBackendExecStatus(const ReportExecStatusRequestPB& req
// Transition the status if we're not already in a terminal state. This won't block
// because either this transitions to an ERROR state or the query is already in
// a terminal state.
discard_result(
UpdateExecState(status, is_fragment_failure ? &failed_instance_id : nullptr,
NetworkAddressPBToString(backend_state->impalad_address())));
// If both 'retryable_status' and 'status' are errors, prefer 'retryable_status' as
// it includes 'status' as well as additional error log information from
// UpdateBlacklistWithAuxErrorInfo.
const Status& update_exec_state_status =
!retryable_status.ok() ? retryable_status : status;
discard_result(UpdateExecState(update_exec_state_status,
is_fragment_failure ? &failed_instance_id : nullptr,
NetworkAddressPBToString(backend_state->impalad_address())));
}
// We've applied all changes from the final status report - notify waiting threads.
discard_result(backend_exec_complete_barrier_->Notify());
@@ -942,19 +973,26 @@ Status Coordinator::UpdateBackendExecStatus(const ReportExecStatusRequestPB& req
}
}
num_completed_backends_->Add(1);
}
} else {
// Iterate through all AuxErrorInfoPB objects, and use each one to possibly blacklist
// any "faulty" nodes.
Status retryable_status = UpdateBlacklistWithAuxErrorInfo(
&aux_error_info, Status::OK(), backend_state);
// Iterate through all AuxErrorInfoPB objects, and use each one to possibly blacklist
// any "faulty" nodes.
UpdateBlacklistWithAuxErrorInfo(&aux_error_info);
// If any nodes were blacklisted, retry the query.
if (!retryable_status.ok()) {
parent_query_driver_->TryQueryRetry(parent_request_state_, &retryable_status);
}
}
// If query execution has terminated, return a cancelled status to force the fragment
// instance to stop executing.
return IsExecuting() ? Status::OK() : Status::CANCELLED;
}
void Coordinator::UpdateBlacklistWithAuxErrorInfo(
vector<AuxErrorInfoPB>* aux_error_info) {
Status Coordinator::UpdateBlacklistWithAuxErrorInfo(
vector<AuxErrorInfoPB>* aux_error_info, const Status& status,
BackendState* backend_state) {
// If the Backend failed due to a RPC failure, blacklist the destination node of
// the failed RPC. Only blacklist one node per ReportExecStatusRequestPB to avoid
// blacklisting nodes too aggressively. Currently, only blacklist the first node
@@ -991,20 +1029,56 @@ void Coordinator::UpdateBlacklistWithAuxErrorInfo(
// A set of RPC related posix error codes that should cause the target node
// of the failed RPC to be blacklisted.
static const set<int32_t> blacklistable_rpc_error_codes = {
ECONNRESET, // 104: Connection reset by peer
ENOTCONN, // 107: Transport endpoint is not connected
ESHUTDOWN, // 108: Cannot send after transport endpoint shutdown
ECONNREFUSED // 111: Connection refused
ECONNREFUSED // 111: Connection refused
};
// If the RPC error code matches any of the 'blacklistable' errors codes, blacklist
// the target executor of the RPC and return.
if (blacklistable_rpc_error_codes.find(rpc_error_info.posix_error_code())
!= blacklistable_rpc_error_codes.end()) {
string src_node_addr =
NetworkAddressPBToString(backend_state->krpc_impalad_address());
string dest_node_addr = NetworkAddressPBToString(dest_node);
VLOG_QUERY << Substitute(
"Blacklisting $0 because a RPC to it failed, query_id=$1", dest_node_addr,
PrintId(query_id()));
Status retryable_status = Status::Expected(
Substitute("RPC from $0 to $1 failed", src_node_addr, dest_node_addr));
retryable_status.MergeStatus(status);
ExecEnv::GetInstance()->cluster_membership_mgr()->BlacklistExecutor(
dest_node_exec_params->be_desc,
Status(Substitute("RPC to $0 failed", NetworkAddressPBToString(dest_node))));
break;
dest_node_exec_params->be_desc, retryable_status);
// Only blacklist one node per report.
return retryable_status;
}
}
}
return Status::OK();
}
void Coordinator::HandleFailedExecRpcs(vector<BackendState*> failed_backend_states) {
DCHECK(!failed_backend_states.empty());
// Create an error based on the Exec RPC failure Status
vector<string> backend_addresses;
for (BackendState* backend_state : failed_backend_states) {
backend_addresses.push_back(
NetworkAddressPBToString(backend_state->krpc_impalad_address()));
}
Status retryable_status = Status::Expected(
Substitute("ExecFInstances RPC to $0 failed", join(backend_addresses, ",")));
for (BackendState* backend_state : failed_backend_states) {
retryable_status.MergeStatus(
FromKuduStatus(backend_state->exec_rpc_status(), "Exec() rpc failed"));
}
// Retry the query
parent_query_driver_->TryQueryRetry(parent_request_state_, &retryable_status);
}
int64_t Coordinator::GetMaxBackendStateLagMs(NetworkAddressPB* address) {

View File

@@ -50,6 +50,7 @@ class FragmentInstanceState;
class MemTracker;
class ObjectPool;
class PlanRootSink;
class QueryDriver;
class QueryResultSet;
class QuerySchedule;
class QueryState;
@@ -244,8 +245,12 @@ class Coordinator { // NOLINT: The member variables could be re-ordered to save
class FilterState;
class FragmentStats;
/// The parent QueryDriver object for this coordinator. The reference is set in the
/// constructor. It always outlives the coordinator.
QueryDriver* parent_query_driver_;
/// The parent ClientRequestState object for this coordinator. The reference is set in
/// the constructor. It always outlives the this coordinator.
/// the constructor. It always outlives the coordinator.
ClientRequestState* parent_request_state_;
/// owned by the ClientRequestState that owns this coordinator
@@ -583,7 +588,16 @@ class Coordinator { // NOLINT: The member variables could be re-ordered to save
/// AuxErrorInfoPB to classify specific nodes as "faulty" and then blacklists them. A
/// node might be considered "faulty" if, for example, a RPC to that node failed, or a
/// fragment on that node failed due to a disk IO error.
void UpdateBlacklistWithAuxErrorInfo(std::vector<AuxErrorInfoPB>* aux_error_info);
/// 'status' is the Status of the given BackendState. 'backend_state' is the
/// BackendState that reported an error.
/// Returns the Status object used when blacklising a backend, or Status::OK if no
/// backends were blacklisted.
Status UpdateBlacklistWithAuxErrorInfo(std::vector<AuxErrorInfoPB>* aux_error_info,
const Status& status, BackendState* backend_state) WARN_UNUSED_RESULT;
/// Called if the Exec RPC to the given vector of BackendStates failed. Currently, just
/// triggers a retry of the query.
void HandleFailedExecRpcs(std::vector<BackendState*> failed_backend_states);
/// Deletes the query-level staging directory.
Status DeleteQueryLevelStagingDir();

View File

@@ -0,0 +1,373 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#include <thrift/protocol/TDebugProtocol.h>
#include "runtime/exec-env.h"
#include "runtime/query-driver.h"
#include "service/client-request-state.h"
#include "service/frontend.h"
#include "service/impala-server.h"
#include "util/debug-util.h"
#include "util/runtime-profile-counters.h"
#include "common/names.h"
namespace impala {
QueryDriver::QueryDriver(ImpalaServer* parent_server) : parent_server_(parent_server) {}
QueryDriver::~QueryDriver() {
DCHECK(finalized_.Load()) << "Finalize() must have been called";
}
void QueryDriver::CreateClientRequestState(const TQueryCtx& query_ctx,
shared_ptr<ImpalaServer::SessionState> session_state, QueryHandle* query_handle) {
DCHECK(exec_request_ == nullptr);
DCHECK(client_request_state_ == nullptr);
ExecEnv* exec_env = ExecEnv::GetInstance();
lock_guard<SpinLock> l(client_request_state_lock_);
exec_request_ = make_unique<TExecRequest>();
client_request_state_ =
make_unique<ClientRequestState>(query_ctx, exec_env->frontend(), parent_server_,
session_state, exec_request_.get(), query_handle->query_driver().get());
DCHECK(query_handle != nullptr);
(*query_handle).SetClientRequestState(client_request_state_.get());
}
Status QueryDriver::RunFrontendPlanner(const TQueryCtx& query_ctx) {
// Takes the TQueryCtx and calls into the frontend to initialize the TExecRequest for
// this query.
DCHECK(client_request_state_ != nullptr);
DCHECK(exec_request_ != nullptr);
RETURN_IF_ERROR(client_request_state_->UpdateQueryStatus(
ExecEnv::GetInstance()->frontend()->GetExecRequest(
query_ctx, exec_request_.get())));
return Status::OK();
}
ClientRequestState* QueryDriver::GetActiveClientRequestState() {
lock_guard<SpinLock> l(client_request_state_lock_);
if (retried_client_request_state_ != nullptr) {
return retried_client_request_state_.get();
}
DCHECK(client_request_state_ != nullptr);
return client_request_state_.get();
}
ClientRequestState* QueryDriver::GetClientRequestState(const TUniqueId& query_id) {
lock_guard<SpinLock> l(client_request_state_lock_);
if (retried_client_request_state_ != nullptr
&& retried_client_request_state_->query_id() == query_id) {
return retried_client_request_state_.get();
}
DCHECK(client_request_state_ != nullptr);
DCHECK(client_request_state_->query_id() == query_id);
return client_request_state_.get();
}
void QueryDriver::TryQueryRetry(
ClientRequestState* client_request_state, Status* error, bool* was_retried) {
DCHECK(error != nullptr);
if (was_retried != nullptr) *was_retried = false;
// Get the most recent query attempt, and retry it.
const TUniqueId& query_id = client_request_state->query_id();
DCHECK(client_request_state->schedule() != nullptr);
if (client_request_state->schedule()->query_options().retry_failed_queries) {
lock_guard<mutex> l(*client_request_state->lock());
// Queries can only be retried if no rows for the query have been fetched
// (IMPALA-9225).
if (client_request_state->fetched_rows()) {
string err_msg = Substitute("Skipping retry of query_id=$0 because the client has "
"already fetched some rows",
PrintId(query_id));
VLOG_QUERY << err_msg;
error->AddDetail(err_msg);
return;
}
// If a retry for the failed query has already been scheduled, don't retry it
// again.
if (client_request_state->WasRetried()) {
return;
}
// Queries can only be retried once (IMPALA-9200).
if (client_request_state->IsRetriedQuery()) {
VLOG_QUERY << Substitute(
"Skipping retry of query_id=$0 because it has already been retried",
PrintId(query_id));
// If query retries are enabled, but the max number of retries has been hit,
// include the number of retries in the error message.
error->AddDetail("Max retry limit was hit. Query was retried 1 time(s).");
return;
}
const TUniqueId& query_id = client_request_state_->query_id();
// Triggering a retry from the INITIALIZED phase is possible: the
// cancellation thread pool can kill a query while it is in the INITIALIZATION phase.
ClientRequestState::ExecState exec_state = client_request_state_->exec_state();
DCHECK(exec_state == ClientRequestState::ExecState::INITIALIZED
|| exec_state == ClientRequestState::ExecState::PENDING
|| exec_state == ClientRequestState::ExecState::RUNNING)
<< Substitute(
"Illegal state: $0", ClientRequestState::ExecStateToString(exec_state));
// If a retry has already been scheduled for this query, do not schedule another one.
DCHECK(client_request_state_->retry_state()
== ClientRequestState::RetryState::NOT_RETRIED)
<< Substitute("Cannot retry a that has already been retried query_id = $0",
PrintId(query_id));
// Update the state and then schedule the retry asynchronously.
client_request_state_->MarkAsRetrying(*error);
// Another reference to this QueryDriver (via the shared_ptr) needs to be created and
// passed to the thread so that a valid shared_ptr exists while the thread is running.
// Otherwise it is possible that the user cancels the query and this QueryDriver gets
// deleted by the shared_ptr.
shared_ptr<QueryDriver> query_driver = parent_server_->GetQueryDriver(query_id);
// Launch the query retry in a separate thread, 'was_retried' is set to true
// if the query retry was successfully launched.
Status status = Thread::Create("impala-server",
Substitute("query-retry-thread-$0", PrintId(query_id)),
&QueryDriver::RetryQueryFromThread, this, *error, query_driver,
&retry_query_thread_);
if (!status.ok()) {
LOG(ERROR) << Substitute(
"Unable to schedule a retry of query $0 due to thread creation error $1",
PrintId(query_id), status.GetDetail());
} else if (was_retried != nullptr) {
*was_retried = true;
}
}
}
void QueryDriver::RetryQueryFromThread(
const Status& error, shared_ptr<QueryDriver> query_driver) {
// This method does not require holding the ClientRequestState::lock_ for the original
// query. This ensures that the client can still interact (e.g. poll the state) of the
// original query while the new query is being created. This is necessary as it might
// take a non-trivial amount of time to setup and start running the new query.
DCHECK(query_driver.get() == this);
const TUniqueId& query_id = client_request_state_->query_id();
VLOG_QUERY << Substitute(
"Retrying query $0 with error message $1", PrintId(query_id), error.GetDetail());
// There should be no retried client request state.
ClientRequestState* request_state;
{
lock_guard<SpinLock> l(client_request_state_lock_);
DCHECK(retried_client_request_state_ == nullptr);
DCHECK(client_request_state_ != nullptr);
request_state = client_request_state_.get();
}
DCHECK(request_state->retry_state() == ClientRequestState::RetryState::RETRYING)
<< Substitute("query=$0 unexpected state $1", PrintId(request_state->query_id()),
request_state->ExecStateToString(request_state->exec_state()));
shared_ptr<ImpalaServer::SessionState> session = request_state->session();
// Cancel the query. 'check_inflight' should be false because (1) a retry can be
// triggered when the query is in the INITIALIZED state, and (2) the user could have
// already cancelled the query.
Status status = request_state->Cancel(false, nullptr);
if (!status.ok()) {
status.AddDetail(Substitute("Failed to retry query $0", PrintId(query_id)));
discard_result(request_state->UpdateQueryStatus(status));
return;
}
unique_ptr<ClientRequestState> retry_request_state = nullptr;
CreateRetriedClientRequestState(request_state, &retry_request_state, &session);
DCHECK(retry_request_state != nullptr);
const TUniqueId& retry_query_id = retry_request_state->query_id();
VLOG_QUERY << Substitute("Retrying query $0 with new query id $1", PrintId(query_id),
PrintId(retry_query_id));
// The steps below mimic what is done when a query is first launched. See
// ImpalaServer::ExecuteStatement.
// Mark the session as active. This is necessary because a ScopedSessionState may not
// actively be opened at this time. A reference to the session (SessionState::ref_count)
// is necessary when calling ImpalaServer::RegisterQuery with the session. Furthermore,
// a reference to the session is necessary to ensure that the session does not get
// expired while the retry is running.
parent_server_->MarkSessionActive(session);
// A QueryHandle instance is required for a few of the methods called below.
QueryHandle retry_query_handle;
retry_query_handle.SetHandle(query_driver, retry_request_state.get());
// Register the new query.
status = parent_server_->RegisterQuery(retry_query_id, session, &retry_query_handle);
if (!status.ok()) {
string error_msg = Substitute(
"RegisterQuery for new query with id $0 failed", PrintId(retry_query_id));
HandleRetryFailure(&status, &error_msg, request_state, retry_query_id);
return;
}
// Run the new query.
status = retry_request_state->Exec();
if (!status.ok()) {
string error_msg =
Substitute("Exec for new query with id $0 failed", PrintId(retry_query_id));
HandleRetryFailure(&status, &error_msg, request_state, retry_query_id);
return;
}
status = retry_request_state->WaitAsync();
if (!status.ok()) {
string error_msg =
Substitute("WaitAsync for new query with id $0 failed", PrintId(retry_query_id));
HandleRetryFailure(&status, &error_msg, request_state, retry_query_id);
return;
}
// Optionally enable result caching on the ClientRequestState. The
// 'result_cache_max_size' value was already validated in ImpalaHs2Server, so it does
// not need to be validated again.
if (request_state->IsResultCacheingEnabled()) {
status = parent_server_->SetupResultsCacheing(
retry_query_handle, session, request_state->result_cache_max_size());
if (!status.ok()) {
string error_msg = Substitute(
"Setting up results cacheing for query $0 failed", PrintId(retry_query_id));
HandleRetryFailure(&status, &error_msg, request_state, retry_query_id);
return;
}
}
// Mark the new query as "in flight".
status = parent_server_->SetQueryInflight(session, retry_query_handle);
if (!status.ok()) {
string error_msg = Substitute(
"SetQueryInFlight for new query with id $0 failed", PrintId(retry_query_id));
HandleRetryFailure(&status, &error_msg, request_state, retry_query_id);
return;
}
// 'client_request_state_' points to the original query and
// 'retried_client_request_state_' points to the retried query.
{
lock_guard<SpinLock> l(client_request_state_lock_);
// Before exposing the new query, check if the original query was unregistered while
// the new query was being created. If it was, then abort the new query.
if (parent_server_->GetQueryDriver(query_id) == nullptr) {
string error_msg = Substitute("Query was unregistered");
HandleRetryFailure(&status, &error_msg, request_state, retry_query_id);
return;
}
retried_client_request_state_ = move(retry_request_state);
}
// Mark the original query as successfully retried.
request_state->MarkAsRetried(retry_query_id);
VLOG_QUERY << Substitute("Retried query $0 with new query id $1", PrintId(query_id),
PrintId(retry_query_id));
// Close the original query.
QueryHandle query_handle;
query_handle.SetHandle(query_driver, request_state);
parent_server_->CloseClientRequestState(query_handle);
parent_server_->MarkSessionInactive(session);
}
void QueryDriver::CreateRetriedClientRequestState(ClientRequestState* request_state,
unique_ptr<ClientRequestState>* retry_request_state,
shared_ptr<ImpalaServer::SessionState>* session) {
parent_server_->PrepareQueryContext(&exec_request_->query_exec_request.query_ctx);
// Create the ClientRequestState for the new query.
const TQueryCtx& query_ctx = exec_request_->query_exec_request.query_ctx;
ExecEnv* exec_env = ExecEnv::GetInstance();
*retry_request_state = make_unique<ClientRequestState>(query_ctx, exec_env->frontend(),
parent_server_, *session, exec_request_.get(), request_state->parent_driver());
(*retry_request_state)->SetOriginalId(request_state->query_id());
(*retry_request_state)
->set_user_profile_access(
(*retry_request_state)->exec_request().user_has_profile_access);
if ((*retry_request_state)->exec_request().__isset.result_set_metadata) {
(*retry_request_state)
->set_result_metadata((*retry_request_state)->exec_request().result_set_metadata);
}
}
void QueryDriver::HandleRetryFailure(Status* status, string* error_msg,
ClientRequestState* request_state, const TUniqueId& retry_query_id) {
status->AddDetail(
Substitute("Failed to retry query $0", PrintId(request_state->query_id())));
status->AddDetail(*error_msg);
discard_result(request_state->UpdateQueryStatus(*status));
parent_server_->UnregisterQueryDiscardResult(retry_query_id, false, status);
}
Status QueryDriver::Finalize(
QueryHandle* query_handle, bool check_inflight, const Status* cause) {
if (!finalized_.CompareAndSwap(false, true)) {
// Return error as-if the query was already unregistered, so that it appears to the
// client as-if unregistration already happened. We don't need a distinct
// client-visible error for this case.
lock_guard<SpinLock> l(client_request_state_lock_);
return Status::Expected(
TErrorCode::INVALID_QUERY_HANDLE, PrintId(client_request_state_->query_id()));
}
RETURN_IF_ERROR((*query_handle)->Finalize(check_inflight, cause));
return Status::OK();
}
Status QueryDriver::Unregister(QueryDriverMap* query_driver_map) {
DCHECK(finalized_.Load());
const TUniqueId* query_id = nullptr;
const TUniqueId* retry_query_id = nullptr;
{
// In order to preserve a consistent lock ordering, client_request_state_lock_ is
// released before DeleteQueryDriver is called, as DeleteQueryDriver requires taking
// a ScopedShardedMapRef (a sharded map lock). Methods in ImpalaServer (such as
// UnresponsiveBackendThread) require taking a ScopedShardedMapRef and then calling
// Get*ClientRequestState methods. So in order to define a consistent lock ordering
// (e.g. acquire ScopedShardedMapRef before client_request_state_lock_)
// client_request_state_lock_ is released before calling DeleteQueryDriver.
lock_guard<SpinLock> l(client_request_state_lock_);
query_id = &client_request_state_->query_id();
if (retried_client_request_state_ != nullptr) {
retry_query_id = &retried_client_request_state_->query_id();
}
}
RETURN_IF_ERROR(query_driver_map->DeleteQueryDriver(*query_id));
if (retry_query_id != nullptr) {
RETURN_IF_ERROR(query_driver_map->DeleteQueryDriver(*retry_query_id));
}
return Status::OK();
}
void QueryDriver::CreateNewDriver(ImpalaServer* impala_server, QueryHandle* query_handle,
const TQueryCtx& query_ctx, shared_ptr<ImpalaServer::SessionState> session_state) {
query_handle->query_driver_ = std::make_shared<QueryDriver>(impala_server);
query_handle->query_driver_->CreateClientRequestState(
query_ctx, session_state, query_handle);
}
}

View File

@@ -0,0 +1,259 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#pragma once
#include "common/status.h"
#include "service/impala-server.h"
#include "common/names.h"
namespace impala {
class ClientRequestState;
class ExecEnv;
class TExecRequest;
/// References to a ClientRequestState should be done via a QueryHandle. A
/// ClientRequestState requires a QueryDriver to be in scope, since the QueryDriver
/// owns the ClientRequestState. The QueryHandle is similar to a unique_ptr in that it
/// wraps a pointer to a given object (in this case ClientRequestState) and overloads
/// the '->' and '*' operators. The ClientRequestState is accessed using the '->' and '*'
/// operators. This ensures that whenever a ClientRequestState is referenced, a valid
/// shared pointer to a QueryDriver is always in scope.
struct QueryHandle {
/// Sets the QueryDriver and ClientRequestState for this handle.
void SetHandle(
std::shared_ptr<QueryDriver> query_driver, ClientRequestState* request_state) {
SetQueryDriver(move(query_driver));
SetClientRequestState(request_state);
}
const std::shared_ptr<QueryDriver>& query_driver() const { return query_driver_; }
inline ClientRequestState* operator->() const {
DCHECK(request_state_ != nullptr);
return request_state_;
}
inline ClientRequestState& operator*() const {
DCHECK(request_state_ != nullptr);
return *request_state_;
}
private:
friend class QueryDriver;
/// Sets the QueryDriver for this handle.
void SetQueryDriver(std::shared_ptr<QueryDriver> query_driver) {
query_driver_ = std::move(query_driver);
}
/// Sets the ClientRequestState for this handle.
void SetClientRequestState(ClientRequestState* request_state) {
request_state_ = request_state;
}
std::shared_ptr<QueryDriver> query_driver_;
ClientRequestState* request_state_ = nullptr;
};
/// The QueryDriver owns the ClientRequestStates for a query. A single query can map
/// to multiple ClientRequestStates if the query is retried multiple times. The
/// QueryDriver sits between the ImpalaServer and ClientRequestState. Currently, it
/// mainly handles query retries.
///
/// ImpalaServer vs. QueryDriver vs. ClientRequestState vs. Coordinator:
///
/// All of these classes work together to run a client-submitted query end to end. A
/// client submits a request to the ImpalaServer to run a query. The ImpalaServer
/// creates a QueryDriver and ClientRequestState to run the query. The
/// ClientRequestState creates a Coordinator that is responsible for coordinating the
/// execution of all query fragments and for tracking the lifecyle of the fragments. A
/// fetch results request flows from the ImpalaServer to the ClientRequestState and then
/// to the Coordinator, which fetches the results from the fragment running on the
/// coordinator process.
///
/// Query Retries:
///
/// Query retries are driven by the 'TryQueryRetry' and 'RetryQueryFromThread' methods.
/// Each query retry creates a new ClientRequestState. In other words, a single
/// ClientRequestState corresponds to a single attempt of a query. Thus, a QueryDriver can
/// own multiple ClientRequestStates, one for each query attempt.
///
/// Retries are done asynchronously in a separate thread. If the query cannot be retried
/// for any reason, the original query is unregistered and moved to the ERROR state. The
/// steps required to launch a retry of a query are very similar to the steps necessary
/// to start the original attempt of the query, except parsing, planning, optimizing,
/// etc. are all skipped. This is done by cacheing the TExecRequest from the original
/// query and re-using it for all query retries.
///
/// At a high level, retrying a query requires performing the following steps:
/// * Cancelling the original query
/// * Creating a new ClientRequestState with a unique query id and a copy of the
/// TExecRequest from the original query
/// * Registering the new query (ImpalaServer::RegisterQuery)
/// * Launching the new query (ClientRequestState::Exec)
/// * Closing and unregistering the original query
///
/// *Transparent* Query Retries:
/// A key feature of query retries is that they should be "transparent" from the client
/// perspective. No client code modifications should be necessary to support query
/// retries. The QueryDriver makes query retries "transparent" by introducing the
/// concept of an "active" ClientRequestState. The "active" ClientRequestState refers
/// to the currently running attempt to run the query, and is accessible by the method
/// GetActiveClientRequestState(). Callers (e.g. ImpalaServer) can use this method to get
/// a reference to the most recent attempt of the query.
///
/// Disclaimer: While query retries are designed to be *transparent* there are scenarios
/// where clients may see some unexpected behavior when a query is retried.
///
/// When a query is retried, the retry is modeled as a brand new query with a new query
/// id, which will be distinct from the query id of the originally submitted query that
/// ultimately failed. So users might see multiple query ids in play during the lifecycle
/// of a single query submission.
///
/// Since a query retry is a brand new query, that query has its own runtime profile as
/// well; the runtime profiles of the failed and retried queries will be linked together.
/// Users need to be aware that there can be multiple runtime profiles associated with a
/// single query submission.
///
/// Thread Safety:
///
/// Only GetClientRequestState(query_id) and GetActiveClientRequestState() are thread
/// safe. They are protected by an internal SpinLock.
class QueryDriver {
public:
QueryDriver(ImpalaServer* parent_server);
~QueryDriver();
/// Creates the TExecRequest for this query. The TExecRequest is owned by the
/// QueryDriver. The TExecRequest is created by calling into Frontend.java,
/// specifically, the Frontend#createExecRequest(PlanCtx) method. When creating the
/// TExecRequest, the Frontend runs the parser, analyzer, authorization code, planner,
/// optimizer, etc. The TQueryCtx is created by the ImpalaServer and contains the full
/// query string (TQueryCtx::TClientRequest::stmt).
Status RunFrontendPlanner(const TQueryCtx& query_ctx) WARN_UNUSED_RESULT;
/// Returns the ClientRequestState corresponding to the given query id.
ClientRequestState* GetClientRequestState(const TUniqueId& query_id);
/// Returns the active ClientRequestState for the query. If a query is retried, this
/// method returns the most recent attempt of the query.
ClientRequestState* GetActiveClientRequestState();
/// Retry the query if query retries are enabled (they are enabled / disabled using the
/// query option 'retry_failed_queries') and if the query can be retried. Queries can
/// only be retried if (1) no rows have already been fetched for the query, (2) the
/// query has not already been retried (when a query fails, only a single retry of that
/// query attempt should be run), and (3) the max number of retries has not been
/// exceeded (currently the limit is just one retry). Queries should only be retried if
/// there has been a cluster membership change. So either a node is blacklisted or a
/// statestore update removes a node from the cluster membership. The retry is done
/// asynchronously by a dedicated thread. 'error' is the reason why the query failed. If
/// the attempt to retry the query failed, additional details might be added to the
/// status. If 'was_retried' is not nullptr it is set to true if the query was actually
/// retried, false otherwise. This method is idempotent, it can safely be called
/// multiple times, however, only the first call to the method will trigger a retry.
/// This method will set the query_status_ of the given client_request_state.
void TryQueryRetry(ClientRequestState* client_request_state, Status* error,
bool* was_retried = nullptr);
/// Finalize this QueryDriver. Must be called before Unregister(...) is called.
/// This indicates that the query should no longer be considered registered from the
/// client's point of view. Returns an INVALID_QUERY_HANDLE error if finalization
/// already started. After this method has been called, finalized() will return true.
/// 'check_inflight' and 'cause' are passed to
/// ClientRequestState::Finalize(bool, Status).
Status Finalize(QueryHandle* query_handle, bool check_inflight, const Status* cause);
/// Delete this query from the given QueryDriverMap.
Status Unregister(QueryDriverMap* query_driver_map) WARN_UNUSED_RESULT;
/// True if Finalize() was called.
bool finalized() const { return finalized_.Load(); }
/// Creates a new QueryDriver instance using the given ImpalaServer. Creates the
/// ClientRequestState for the given 'query_ctx' and 'session_state'. Sets the given
/// QueryHandle's QueryDriver.
static void CreateNewDriver(ImpalaServer* impala_server, QueryHandle* query_handle,
const TQueryCtx& query_ctx,
std::shared_ptr<ImpalaServer::SessionState> session_state);
private:
/// Helper method to process query retries, called by the 'retry_query_thread_'.
/// 'error' is the reason why the query failed. The failed query is cancelled, and then
/// a new ClientRequestState is created for the retried query. The new
/// ClientRequestState copies the TExecRequest from the failed query in order to avoid
/// query compilation and planning again. Once the new query is registered and launched,
/// the failed query is unregistered. 'query_driver' is a shared_ptr to 'this'
/// QueryDriver. The pointer is necessary to ensure that 'this' QueryDriver is not
/// deleted while the thread is running.
void RetryQueryFromThread(
const Status& error, std::shared_ptr<QueryDriver> query_driver);
/// Creates the initial ClientRequestState for the given TQueryCtx. Should only be
/// called once by the ImpalaServer. Additional ClientRequestStates are created by
/// CreateRetriedClientRequestState, although they are only created if the query
/// is retried.
void CreateClientRequestState(const TQueryCtx& query_ctx,
std::shared_ptr<ImpalaServer::SessionState> session_state,
QueryHandle* query_handle);
/// Helper method for RetryQueryFromThread. Creates the retry client request state (the
/// new attempt of the query) based on the original request state. Uses the TExecRequest
/// from the original request state to create the retry request state. Creates a new
/// query id for the retry request state.
void CreateRetriedClientRequestState(ClientRequestState* request_state,
std::unique_ptr<ClientRequestState>* retry_request_state,
std::shared_ptr<ImpalaServer::SessionState>* session);
/// Helper method for handling failures when retrying a query. 'status' is the reason
/// why the retry failed and is expected to be in the error state. Additional details
/// are added to the 'status'. Once the 'status' has been updated, it is set as the
/// 'query status' of the given 'request_state'. Finally, the 'request_state' is
/// unregistered from 'parent_server_', since the retry failed.
void HandleRetryFailure(Status* status, string* error_msg,
ClientRequestState* request_state, const TUniqueId& retry_query_id);
/// ImpalaServer that owns this QueryDriver.
ImpalaServer* parent_server_;
/// Protects 'client_request_state_' and 'retried_client_request_state_'.
SpinLock client_request_state_lock_;
/// The ClientRequestState for the query. Set in 'CreateClientRequestState'. Owned by
/// the QueryDriver.
std::unique_ptr<ClientRequestState> client_request_state_;
/// The ClientRequestState for the retried query. Set in 'RetryQueryFromThread'. Only
/// set if the query is retried. Owned by the QueryDriver.
std::unique_ptr<ClientRequestState> retried_client_request_state_;
/// The TExecRequest for the query. Created in 'CreateClientRequestState' and loaded in
/// 'RunFrontendPlanner'.
std::unique_ptr<TExecRequest> exec_request_;
/// Thread to process query retry requests. Done in a separate thread to avoid blocking
/// control service RPC threads.
std::unique_ptr<Thread> retry_query_thread_;
/// True if a thread has called Finalize(). Threads calling Finalize() do a
/// compare-and-swap on this so that only one thread can proceed.
AtomicBool finalized_{false};
};
}

View File

@@ -28,7 +28,6 @@ set_source_files_properties(${DATA_STREAM_SERVICE_PROTO_SRCS} PROPERTIES GENERAT
add_library(Service
child-query.cc
client-request-state.cc
client-request-state-map.cc
${CONTROL_SERVICE_PROTO_SRCS}
control-service.cc
${DATA_STREAM_SERVICE_PROTO_SRCS}
@@ -42,6 +41,7 @@ add_library(Service
impala-internal-service.cc
impalad-main.cc
impala-server.cc
query-driver-map.cc
query-options.cc
query-result-set.cc
)

View File

@@ -38,6 +38,7 @@
#include "runtime/mem-tracker.h"
#include "runtime/row-batch.h"
#include "runtime/runtime-state.h"
#include "runtime/query-driver.h"
#include "scheduling/admission-controller.h"
#include "scheduling/scheduler.h"
#include "service/frontend.h"
@@ -90,13 +91,15 @@ static const string TABLES_MISSING_STATS_KEY = "Tables Missing Stats";
static const string TABLES_WITH_CORRUPT_STATS_KEY = "Tables With Corrupt Table Stats";
static const string TABLES_WITH_MISSING_DISK_IDS_KEY = "Tables With Missing Disk Ids";
ClientRequestState::ClientRequestState(
const TQueryCtx& query_ctx, ExecEnv* exec_env, Frontend* frontend,
ImpalaServer* server, shared_ptr<ImpalaServer::SessionState> session)
static const string QUERY_STATUS_KEY = "Query Status";
static const string RETRY_STATUS_KEY = "Retry Status";
ClientRequestState::ClientRequestState(const TQueryCtx& query_ctx, Frontend* frontend,
ImpalaServer* server, shared_ptr<ImpalaServer::SessionState> session,
TExecRequest* exec_request, QueryDriver* query_driver)
: query_ctx_(query_ctx),
last_active_time_ms_(numeric_limits<int64_t>::max()),
child_query_executor_(new ChildQueryExecutor),
exec_env_(exec_env),
session_(session),
coord_exec_called_(false),
// Profile is assigned name w/ id after planning
@@ -104,10 +107,12 @@ ClientRequestState::ClientRequestState(
frontend_profile_(RuntimeProfile::Create(&profile_pool_, "Frontend")),
server_profile_(RuntimeProfile::Create(&profile_pool_, "ImpalaServer")),
summary_profile_(RuntimeProfile::Create(&profile_pool_, "Summary")),
exec_request_(exec_request),
frontend_(frontend),
parent_server_(server),
start_time_us_(UnixMicros()),
fetch_rows_timeout_us_(MICROS_PER_MILLI * query_options().fetch_rows_timeout_ms) {
fetch_rows_timeout_us_(MICROS_PER_MILLI * query_options().fetch_rows_timeout_ms),
parent_driver_(query_driver) {
#ifndef NDEBUG
profile_->AddInfoString("DEBUG MODE WARNING", "Query profile created while running a "
"DEBUG build of Impala. Use RELEASE builds to measure query performance.");
@@ -140,7 +145,7 @@ ClientRequestState::ClientRequestState(
summary_profile_->AddInfoString("Query Type", "N/A");
summary_profile_->AddInfoString("Query State", PrintThriftEnum(BeeswaxQueryState()));
summary_profile_->AddInfoString(
"Impala Query State", ExecStateToString(exec_state_.Load()));
"Impala Query State", ExecStateToString(exec_state()));
summary_profile_->AddInfoString("Query Status", "OK");
summary_profile_->AddInfoString("Impala Version", GetVersionString(/* compact */ true));
summary_profile_->AddInfoString("User", effective_user());
@@ -152,14 +157,13 @@ ClientRequestState::ClientRequestState(
summary_profile_->AddInfoStringRedacted(
"Sql Statement", query_ctx_.client_request.stmt);
summary_profile_->AddInfoString("Coordinator",
TNetworkAddressToString(exec_env->GetThriftBackendAddress()));
TNetworkAddressToString(ExecEnv::GetInstance()->GetThriftBackendAddress()));
summary_profile_->AddChild(frontend_profile_);
}
ClientRequestState::~ClientRequestState() {
DCHECK(wait_thread_.get() == NULL) << "BlockOnWait() needs to be called!";
DCHECK(started_finalize_.Load()) << "Finalize() must have been called";
}
Status ClientRequestState::SetResultCache(QueryResultSet* cache,
@@ -191,68 +195,68 @@ Status ClientRequestState::Exec() {
summary_profile_->AddInfoString("Query Options (set by configuration)",
DebugQueryOptions(query_ctx_.client_request.query_options));
summary_profile_->AddInfoString("Query Options (set by configuration and planner)",
DebugQueryOptions(exec_request_.query_options));
DebugQueryOptions(exec_request_->query_options));
switch (exec_request_.stmt_type) {
switch (exec_request_->stmt_type) {
case TStmtType::QUERY:
case TStmtType::DML:
DCHECK(exec_request_.__isset.query_exec_request);
RETURN_IF_ERROR(ExecAsyncQueryOrDmlRequest(exec_request_.query_exec_request));
DCHECK(exec_request_->__isset.query_exec_request);
RETURN_IF_ERROR(ExecAsyncQueryOrDmlRequest(exec_request_->query_exec_request));
break;
case TStmtType::EXPLAIN: {
request_result_set_.reset(new vector<TResultRow>(
exec_request_.explain_result.results));
exec_request_->explain_result.results));
break;
}
case TStmtType::TESTCASE: {
DCHECK(exec_request_.__isset.testcase_data_path);
SetResultSet(vector<string>(1, exec_request_.testcase_data_path));
DCHECK(exec_request_->__isset.testcase_data_path);
SetResultSet(vector<string>(1, exec_request_->testcase_data_path));
break;
}
case TStmtType::DDL: {
DCHECK(exec_request_.__isset.catalog_op_request);
DCHECK(exec_request_->__isset.catalog_op_request);
LOG_AND_RETURN_IF_ERROR(ExecDdlRequest());
break;
}
case TStmtType::LOAD: {
DCHECK(exec_request_.__isset.load_data_request);
DCHECK(exec_request_->__isset.load_data_request);
TLoadDataResp response;
RETURN_IF_ERROR(
frontend_->LoadData(exec_request_.load_data_request, &response));
frontend_->LoadData(exec_request_->load_data_request, &response));
request_result_set_.reset(new vector<TResultRow>);
request_result_set_->push_back(response.load_summary);
// Now refresh the table metadata.
TCatalogOpRequest reset_req;
reset_req.__set_sync_ddl(exec_request_.query_options.sync_ddl);
reset_req.__set_sync_ddl(exec_request_->query_options.sync_ddl);
reset_req.__set_op_type(TCatalogOpType::RESET_METADATA);
reset_req.__set_reset_metadata_params(TResetMetadataRequest());
reset_req.reset_metadata_params.__set_header(TCatalogServiceRequestHeader());
reset_req.reset_metadata_params.__set_is_refresh(true);
reset_req.reset_metadata_params.__set_table_name(
exec_request_.load_data_request.table_name);
if (exec_request_.load_data_request.__isset.partition_spec) {
exec_request_->load_data_request.table_name);
if (exec_request_->load_data_request.__isset.partition_spec) {
reset_req.reset_metadata_params.__set_partition_spec(
exec_request_.load_data_request.partition_spec);
exec_request_->load_data_request.partition_spec);
}
reset_req.reset_metadata_params.__set_sync_ddl(
exec_request_.query_options.sync_ddl);
exec_request_->query_options.sync_ddl);
catalog_op_executor_.reset(
new CatalogOpExecutor(exec_env_, frontend_, server_profile_));
new CatalogOpExecutor(ExecEnv::GetInstance(), frontend_, server_profile_));
RETURN_IF_ERROR(catalog_op_executor_->Exec(reset_req));
RETURN_IF_ERROR(parent_server_->ProcessCatalogUpdateResult(
*catalog_op_executor_->update_catalog_result(),
exec_request_.query_options.sync_ddl));
exec_request_->query_options.sync_ddl));
break;
}
case TStmtType::SET: {
DCHECK(exec_request_.__isset.set_query_option_request);
DCHECK(exec_request_->__isset.set_query_option_request);
lock_guard<mutex> l(session_->lock);
if (exec_request_.set_query_option_request.__isset.key) {
if (exec_request_->set_query_option_request.__isset.key) {
// "SET key=value" updates the session query options.
DCHECK(exec_request_.set_query_option_request.__isset.value);
const auto& key = exec_request_.set_query_option_request.key;
const auto& value = exec_request_.set_query_option_request.value;
DCHECK(exec_request_->set_query_option_request.__isset.value);
const auto& key = exec_request_->set_query_option_request.key;
const auto& value = exec_request_->set_query_option_request.value;
RETURN_IF_ERROR(SetQueryOption(key, value, &session_->set_query_options,
&session_->set_query_options_mask));
SetResultSet({}, {}, {});
@@ -265,19 +269,19 @@ Status ClientRequestState::Exec() {
}
} else {
// "SET" or "SET ALL"
bool is_set_all = exec_request_.set_query_option_request.__isset.is_set_all &&
exec_request_.set_query_option_request.is_set_all;
bool is_set_all = exec_request_->set_query_option_request.__isset.is_set_all &&
exec_request_->set_query_option_request.is_set_all;
PopulateResultForSet(is_set_all);
}
break;
}
case TStmtType::ADMIN_FN:
DCHECK(exec_request_.admin_request.type == TAdminRequestType::SHUTDOWN);
DCHECK(exec_request_->admin_request.type == TAdminRequestType::SHUTDOWN);
RETURN_IF_ERROR(ExecShutdownRequest());
break;
default:
stringstream errmsg;
errmsg << "Unknown exec request stmt type: " << exec_request_.stmt_type;
errmsg << "Unknown exec request stmt type: " << exec_request_->stmt_type;
return Status(errmsg.str());
}
@@ -314,7 +318,7 @@ Status ClientRequestState::ExecLocalCatalogOp(
switch (catalog_op.op_type) {
case TCatalogOpType::USE: {
lock_guard<mutex> l(session_->lock);
session_->database = exec_request_.catalog_op_request.use_db_params.db;
session_->database = exec_request_->catalog_op_request.use_db_params.db;
return Status::OK();
}
case TCatalogOpType::SHOW_TABLES: {
@@ -513,13 +517,13 @@ Status ClientRequestState::ExecAsyncQueryOrDmlRequest(
}
void ClientRequestState::FinishExecQueryOrDmlRequest() {
DebugActionNoFail(exec_request_.query_options, "CRS_BEFORE_ADMISSION");
DebugActionNoFail(exec_request_->query_options, "CRS_BEFORE_ADMISSION");
DCHECK(exec_env_->admission_controller() != nullptr);
DCHECK(exec_request_.__isset.query_exec_request);
DCHECK(ExecEnv::GetInstance()->admission_controller() != nullptr);
DCHECK(exec_request_->__isset.query_exec_request);
Status admit_status =
ExecEnv::GetInstance()->admission_controller()->SubmitForAdmission(
{query_id(), exec_request_.query_exec_request, exec_request_.query_options,
{query_id(), exec_request_->query_exec_request, exec_request_->query_options,
summary_profile_, query_events_},
&admit_outcome_, &schedule_);
{
@@ -577,14 +581,14 @@ Status ClientRequestState::ExecDdlRequest() {
if (catalog_op_type() != TCatalogOpType::DDL &&
catalog_op_type() != TCatalogOpType::RESET_METADATA) {
Status status = ExecLocalCatalogOp(exec_request_.catalog_op_request);
Status status = ExecLocalCatalogOp(exec_request_->catalog_op_request);
lock_guard<mutex> l(lock_);
return UpdateQueryStatus(status);
}
if (ddl_type() == TDdlType::COMPUTE_STATS) {
TComputeStatsParams& compute_stats_params =
exec_request_.catalog_op_request.ddl_params.compute_stats_params;
exec_request_->catalog_op_request.ddl_params.compute_stats_params;
RuntimeProfile* child_profile =
RuntimeProfile::Create(&profile_pool_, "Child Queries");
profile_->AddChild(child_profile);
@@ -613,9 +617,9 @@ Status ClientRequestState::ExecDdlRequest() {
return Status::OK();
}
catalog_op_executor_.reset(new CatalogOpExecutor(exec_env_, frontend_,
server_profile_));
Status status = catalog_op_executor_->Exec(exec_request_.catalog_op_request);
catalog_op_executor_.reset(
new CatalogOpExecutor(ExecEnv::GetInstance(), frontend_, server_profile_));
Status status = catalog_op_executor_->Exec(exec_request_->catalog_op_request);
{
lock_guard<mutex> l(lock_);
RETURN_IF_ERROR(UpdateQueryStatus(status));
@@ -628,7 +632,7 @@ Status ClientRequestState::ExecDdlRequest() {
if (catalog_op_type() == TCatalogOpType::DDL &&
ddl_type() == TDdlType::CREATE_TABLE_AS_SELECT &&
!catalog_op_executor_->ddl_exec_response()->new_table_created) {
DCHECK(exec_request_.catalog_op_request.
DCHECK(exec_request_->catalog_op_request.
ddl_params.create_table_params.if_not_exists);
return Status::OK();
}
@@ -636,7 +640,7 @@ Status ClientRequestState::ExecDdlRequest() {
// Add newly created table to catalog cache.
RETURN_IF_ERROR(parent_server_->ProcessCatalogUpdateResult(
*catalog_op_executor_->update_catalog_result(),
exec_request_.query_options.sync_ddl));
exec_request_->query_options.sync_ddl));
if (catalog_op_type() == TCatalogOpType::DDL &&
ddl_type() == TDdlType::CREATE_TABLE_AS_SELECT) {
@@ -644,8 +648,8 @@ Status ClientRequestState::ExecDdlRequest() {
// like a normal DML request. As with other DML requests, it will
// wait for another catalog update if any partitions were altered as a result
// of the operation.
DCHECK(exec_request_.__isset.query_exec_request);
RETURN_IF_ERROR(ExecAsyncQueryOrDmlRequest(exec_request_.query_exec_request));
DCHECK(exec_request_->__isset.query_exec_request);
RETURN_IF_ERROR(ExecAsyncQueryOrDmlRequest(exec_request_->query_exec_request));
}
// Set the results to be reported to the client.
@@ -654,7 +658,7 @@ Status ClientRequestState::ExecDdlRequest() {
}
Status ClientRequestState::ExecShutdownRequest() {
const TShutdownParams& request = exec_request_.admin_request.shutdown_params;
const TShutdownParams& request = exec_request_->admin_request.shutdown_params;
bool backend_port_specified = request.__isset.backend && request.backend.port != 0;
int port = backend_port_specified ? request.backend.port : FLAGS_krpc_port;
// Use the local shutdown code path if the host is unspecified or if it exactly matches
@@ -726,12 +730,6 @@ Status ClientRequestState::ExecShutdownRequest() {
}
Status ClientRequestState::Finalize(bool check_inflight, const Status* cause) {
if (!started_finalize_.CompareAndSwap(false, true)) {
// Return error as-if the query was already unregistered, so that it appears to the
// client as-if unregistration already happened. We don't need a distinct
// client-visible error for this case.
return Status::Expected(TErrorCode::INVALID_QUERY_HANDLE, PrintId(query_id()));
}
RETURN_IF_ERROR(Cancel(check_inflight, cause));
MarkActive();
// Make sure we join on wait_thread_ before we finish (and especially before this object
@@ -850,8 +848,8 @@ void ClientRequestState::Wait() {
UpdateNonErrorExecState(ExecState::FINISHED);
}
// UpdateQueryStatus() or UpdateNonErrorExecState() have updated exec_state_.
ExecState exec_state = exec_state_.Load();
DCHECK(exec_state == ExecState::FINISHED || exec_state == ExecState::ERROR);
DCHECK(exec_state() == ExecState::FINISHED || exec_state() == ExecState::ERROR
|| retry_state() == RetryState::RETRYING || retry_state() == RetryState::RETRIED);
// Notify all the threads blocked on Wait() to finish and then log the query events,
// if any.
{
@@ -864,7 +862,7 @@ void ClientRequestState::Wait() {
Status ClientRequestState::WaitInternal() {
// Explain requests have already populated the result set. Nothing to do here.
if (exec_request_.stmt_type == TStmtType::EXPLAIN) {
if (exec_request_->stmt_type == TStmtType::EXPLAIN) {
MarkInactive();
return Status::OK();
}
@@ -941,7 +939,7 @@ Status ClientRequestState::RestartFetch() {
void ClientRequestState::UpdateNonErrorExecState(ExecState new_state) {
lock_guard<mutex> l(lock_);
ExecState old_state = exec_state_.Load();
ExecState old_state = exec_state();
static string error_msg = "Illegal state transition: $0 -> $1, query_id=$3";
switch (new_state) {
case ExecState::PENDING:
@@ -981,12 +979,31 @@ void ClientRequestState::UpdateNonErrorExecState(ExecState new_state) {
}
}
void ClientRequestState::SetOriginalId(const TUniqueId& original_id) {
// Copy the TUniqueId query_id from the original query.
original_id_ = make_unique<TUniqueId>(original_id);
summary_profile_->AddInfoString("Original Query Id", PrintId(*original_id_));
}
void ClientRequestState::MarkAsRetrying(const Status& status) {
retry_state_.Store(RetryState::RETRYING);
summary_profile_->AddInfoString(
RETRY_STATUS_KEY, RetryStateToString(RetryState::RETRYING));
// Set the query status.
query_status_ = status;
summary_profile_->AddInfoStringRedacted(QUERY_STATUS_KEY, query_status_.GetDetail());
// The Query Status might be overwritten later if the retry fails. "Retry Cause"
// preserves the original error that triggered the retry.
summary_profile_->AddInfoStringRedacted("Retry Cause", query_status_.GetDetail());
}
Status ClientRequestState::UpdateQueryStatus(const Status& status) {
// Preserve the first non-ok status
if (!status.ok() && query_status_.ok()) {
UpdateExecState(ExecState::ERROR);
query_status_ = status;
summary_profile_->AddInfoStringRedacted("Query Status", query_status_.GetDetail());
summary_profile_->AddInfoStringRedacted(QUERY_STATUS_KEY, query_status_.GetDetail());
}
return status;
@@ -996,7 +1013,7 @@ Status ClientRequestState::FetchRowsInternal(const int32_t max_rows,
QueryResultSet* fetched_rows, int64_t block_on_wait_time_us) {
// Wait() guarantees that we've transitioned at least to FINISHED state (and any
// state beyond that should have a non-OK query_status_ set).
DCHECK(exec_state_.Load() == ExecState::FINISHED);
DCHECK(exec_state() == ExecState::FINISHED);
if (eos_.Load()) return Status::OK();
@@ -1137,12 +1154,13 @@ Status ClientRequestState::Cancel(bool check_inflight, const Status* cause) {
{
lock_guard<mutex> lock(lock_);
// If the query has reached a terminal state, no need to update the state.
bool already_done = eos_.Load() || exec_state_.Load() == ExecState::ERROR;
bool already_done = eos_.Load() || exec_state() == ExecState::ERROR;
if (!already_done && cause != NULL) {
DCHECK(!cause->ok());
discard_result(UpdateQueryStatus(*cause));
query_events_->MarkEvent("Cancelled");
DCHECK(exec_state_.Load() == ExecState::ERROR);
DCHECK(exec_state() == ExecState::ERROR
|| retry_state() == RetryState::RETRYING);
}
admit_outcome_.Set(AdmissionOutcome::CANCELLED);
@@ -1162,19 +1180,19 @@ Status ClientRequestState::Cancel(bool check_inflight, const Status* cause) {
}
Status ClientRequestState::UpdateCatalog() {
if (!exec_request_.__isset.query_exec_request ||
exec_request_.query_exec_request.stmt_type != TStmtType::DML) {
if (!exec_request_->__isset.query_exec_request ||
exec_request_->query_exec_request.stmt_type != TStmtType::DML) {
return Status::OK();
}
query_events_->MarkEvent("DML data written");
SCOPED_TIMER(ADD_TIMER(server_profile_, "MetastoreUpdateTimer"));
TQueryExecRequest query_exec_request = exec_request_.query_exec_request;
TQueryExecRequest query_exec_request = exec_request_->query_exec_request;
if (query_exec_request.__isset.finalize_params) {
const TFinalizeParams& finalize_params = query_exec_request.finalize_params;
TUpdateCatalogRequest catalog_update;
catalog_update.__set_sync_ddl(exec_request_.query_options.sync_ddl);
catalog_update.__set_sync_ddl(exec_request_->query_options.sync_ddl);
catalog_update.__set_header(TCatalogServiceRequestHeader());
catalog_update.header.__set_requesting_user(effective_user());
catalog_update.header.__set_client_ip(session()->network_address.hostname);
@@ -1203,7 +1221,7 @@ Status ClientRequestState::UpdateCatalog() {
const TNetworkAddress& address =
MakeNetworkAddress(FLAGS_catalog_service_host, FLAGS_catalog_service_port);
CatalogServiceConnection client(
exec_env_->catalogd_client_cache(), address, &cnxn_status);
ExecEnv::GetInstance()->catalogd_client_cache(), address, &cnxn_status);
RETURN_IF_ERROR(cnxn_status);
VLOG_QUERY << "Executing FinalizeDml() using CatalogService";
@@ -1229,7 +1247,7 @@ Status ClientRequestState::UpdateCatalog() {
query_events_->MarkEvent("Transaction committed");
}
RETURN_IF_ERROR(parent_server_->ProcessCatalogUpdateResult(resp.result,
exec_request_.query_options.sync_ddl));
exec_request_->query_options.sync_ddl));
}
}
query_events_->MarkEvent("DML Metastore update finished");
@@ -1338,7 +1356,7 @@ Status ClientRequestState::UpdateTableAndColumnStats(
DCHECK_GE(child_queries.size(), 1);
DCHECK_LE(child_queries.size(), 2);
catalog_op_executor_.reset(
new CatalogOpExecutor(exec_env_, frontend_, server_profile_));
new CatalogOpExecutor(ExecEnv::GetInstance(), frontend_, server_profile_));
// If there was no column stats query, pass in empty thrift structures to
// ExecComputeStats(). Otherwise pass in the column stats result.
@@ -1350,7 +1368,7 @@ Status ClientRequestState::UpdateTableAndColumnStats(
}
Status status = catalog_op_executor_->ExecComputeStats(
exec_request_.catalog_op_request,
exec_request_->catalog_op_request,
child_queries[0]->result_schema(),
child_queries[0]->result_data(),
col_stats_schema,
@@ -1361,7 +1379,7 @@ Status ClientRequestState::UpdateTableAndColumnStats(
}
RETURN_IF_ERROR(parent_server_->ProcessCatalogUpdateResult(
*catalog_op_executor_->update_catalog_result(),
exec_request_.query_options.sync_ddl));
exec_request_->query_options.sync_ddl));
// Set the results to be reported to the client.
SetResultSet(catalog_op_executor_->ddl_exec_response());
@@ -1389,9 +1407,8 @@ void ClientRequestState::UpdateExecState(ExecState exec_state) {
summary_profile_->AddInfoString("Impala Query State", ExecStateToString(exec_state));
}
apache::hive::service::cli::thrift::TOperationState::type
ClientRequestState::TOperationState() const {
switch (exec_state_.Load()) {
TOperationState::type ClientRequestState::TOperationState() const {
switch (exec_state()) {
case ExecState::INITIALIZED: return TOperationState::INITIALIZED_STATE;
case ExecState::PENDING: return TOperationState::PENDING_STATE;
case ExecState::RUNNING: return TOperationState::RUNNING_STATE;
@@ -1405,7 +1422,7 @@ ClientRequestState::TOperationState() const {
}
beeswax::QueryState::type ClientRequestState::BeeswaxQueryState() const {
switch (exec_state_.Load()) {
switch (exec_state()) {
case ExecState::INITIALIZED: return beeswax::QueryState::CREATED;
case ExecState::PENDING: return beeswax::QueryState::COMPILED;
case ExecState::RUNNING: return beeswax::QueryState::RUNNING;
@@ -1448,9 +1465,26 @@ bool ClientRequestState::GetDmlStats(TDmlResult* dml_result, Status* query_statu
return true;
}
Status ClientRequestState::InitExecRequest(const TQueryCtx& query_ctx) {
return UpdateQueryStatus(
exec_env_->frontend()->GetExecRequest(query_ctx, &exec_request_));
void ClientRequestState::WaitUntilRetried() {
unique_lock<mutex> l(lock_);
DCHECK(retry_state() != RetryState::NOT_RETRIED);
while (retry_state() == RetryState::RETRYING) {
block_until_retried_cv_.Wait(l);
}
DCHECK(retry_state() == RetryState::RETRIED
|| exec_state() == ExecState::ERROR);
}
void ClientRequestState::MarkAsRetried(const TUniqueId& retried_id) {
DCHECK(retry_state() == RetryState::RETRYING)
<< Substitute("Illegal retry state transition: $0 -> RETRYING, query_id=$2",
RetryStateToString(retry_state()), PrintId(query_id()));
retry_state_.Store(RetryState::RETRIED);
summary_profile_->AddInfoStringRedacted(
RETRY_STATUS_KEY, RetryStateToString(RetryState::RETRIED));
summary_profile_->AddInfoString("Retried Query Id", PrintId(retried_id));
UpdateExecState(ExecState::ERROR);
block_until_retried_cv_.NotifyOne();
}
const string& ClientRequestState::effective_user() const {
@@ -1469,11 +1503,11 @@ void ClientRequestState::UpdateEndTime() {
int64_t ClientRequestState::GetTransactionId() const {
DCHECK(InTransaction());
return exec_request_.query_exec_request.finalize_params.transaction_id;
return exec_request_->query_exec_request.finalize_params.transaction_id;
}
bool ClientRequestState::InTransaction() const {
return exec_request_.query_exec_request.finalize_params.__isset.transaction_id &&
return exec_request_->query_exec_request.finalize_params.__isset.transaction_id &&
!transaction_closed_;
}
@@ -1531,7 +1565,7 @@ void ClientRequestState::LogQueryEvents() {
}
Status ClientRequestState::LogAuditRecord(const Status& query_status) {
const TExecRequest& request = exec_request_;
const TExecRequest& request = exec_request();
stringstream ss;
rapidjson::StringBuffer buffer;
rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
@@ -1607,7 +1641,7 @@ Status ClientRequestState::LogAuditRecord(const Status& query_status) {
}
Status ClientRequestState::LogLineageRecord() {
const TExecRequest& request = exec_request_;
const TExecRequest& request = exec_request();
if (request.stmt_type == TStmtType::EXPLAIN || (!request.__isset.query_exec_request &&
!request.__isset.catalog_op_request)) {
return Status::OK();
@@ -1654,7 +1688,7 @@ Status ClientRequestState::LogLineageRecord() {
// invoke QueryEventHooks
TQueryCompleteContext query_complete_context;
query_complete_context.__set_lineage_string(lineage_record);
const Status& status = exec_env_->frontend()->CallQueryCompleteHooks(
const Status& status = ExecEnv::GetInstance()->frontend()->CallQueryCompleteHooks(
query_complete_context);
if (!status.ok()) {
@@ -1684,7 +1718,7 @@ Status ClientRequestState::LogLineageRecord() {
return Status::OK();
}
string ClientRequestState::ExecStateToString(ExecState state) const {
string ClientRequestState::ExecStateToString(ExecState state) {
static const unordered_map<ClientRequestState::ExecState, const char*>
exec_state_strings{{ClientRequestState::ExecState::INITIALIZED, "INITIALIZED"},
{ClientRequestState::ExecState::PENDING, "PENDING"},
@@ -1693,4 +1727,12 @@ string ClientRequestState::ExecStateToString(ExecState state) const {
{ClientRequestState::ExecState::ERROR, "ERROR"}};
return exec_state_strings.at(state);
}
string ClientRequestState::RetryStateToString(RetryState state) {
static const unordered_map<ClientRequestState::RetryState, const char*>
retry_state_strings{{ClientRequestState::RetryState::NOT_RETRIED, "NOT_RETRIED"},
{ClientRequestState::RetryState::RETRYING, "RETRYING"},
{ClientRequestState::RetryState::RETRIED, "RETRIED"}};
return retry_state_strings.at(state);
}
}

View File

@@ -37,7 +37,6 @@ namespace impala {
class ClientRequestStateCleaner;
class Coordinator;
class ExecEnv;
class Expr;
class Frontend;
class ReportExecStatusRequestPB;
@@ -65,19 +64,26 @@ enum class AdmissionOutcome;
/// the TStmtType). Successful QUERY / DML queries transition from INITIALIZED to PENDING
/// to RUNNING to FINISHED whereas DDL queries skip the PENDING phase.
///
/// Retry State:
/// The retry state is only used when the query corresponding to this ClientRequestState
/// is retried. Otherwise, the default state is NOT_RETRIED. MarkAsRetrying and
/// MarkAsRetried set the state to RETRYING and RETRIED, respectively. Queries can only
/// transition from NOT_RETRIED to RETRYING and then finally to RETRIED.
///
/// TODO: Compute stats is the only stmt that requires child queries. Once the
/// CatalogService performs background stats gathering the concept of child queries
/// will likely become obsolete. Remove all child-query related code from this class.
class ClientRequestState {
public:
ClientRequestState(const TQueryCtx& query_ctx, ExecEnv* exec_env, Frontend* frontend,
ImpalaServer* server, std::shared_ptr<ImpalaServer::SessionState> session);
ClientRequestState(const TQueryCtx& query_ctx, Frontend* frontend, ImpalaServer* server,
std::shared_ptr<ImpalaServer::SessionState> session, TExecRequest* exec_request,
QueryDriver* query_driver);
~ClientRequestState();
enum class ExecState {
INITIALIZED, PENDING, RUNNING, FINISHED, ERROR
};
enum class ExecState { INITIALIZED, PENDING, RUNNING, FINISHED, ERROR };
enum class RetryState { RETRYING, RETRIED, NOT_RETRIED };
/// Sets the profile that is produced by the frontend. The frontend creates the
/// profile during planning and returns it to the backend via TExecRequest,
@@ -162,7 +168,7 @@ class ClientRequestState {
Status UpdateQueryStatus(const Status& status) WARN_UNUSED_RESULT;
/// Cancels the child queries and the coordinator with the given cause.
/// If cause is NULL, it assume this was deliberately cancelled by the user while in
/// If cause is NULL, it assumes this was deliberately cancelled by the user while in
/// FINISHED state. Otherwise, sets state to ERROR (TODO: IMPALA-1262: use CANCELLED).
/// Does nothing if the query has reached EOS or already cancelled.
///
@@ -200,14 +206,19 @@ class ClientRequestState {
/// Caller must not hold 'lock()'.
bool GetDmlStats(TDmlResult* dml_result, Status* query_status);
/// Creates and sets the TExecRequest for the query associated with this
/// ClientRequestState. The TExecRequest is created by the Impala frontend via the
/// method Frontend::GetExecRequest(TQueryCtx, TExecRequest). The TQueryCtx is created
/// by the ImpalaServer and contains the full query string
/// (TQueryCtx::TClientRequest::stmt).
Status InitExecRequest(const TQueryCtx& query_ctx);
/// Blocks until this query has been retried. Waits until the ExecState has transitioned
/// to RETRIED (e.g. once MarkAsRetried() has been called). Can only be called if the
/// current state is either RETRIED or RETRYING. Takes lock_.
void WaitUntilRetried();
ImpalaServer::SessionState* session() const { return session_.get(); }
/// Converts the given ExecState to a string representation.
static std::string ExecStateToString(ExecState state);
/// Converts the given RetryState to a string representation.
static std::string RetryStateToString(RetryState state);
/// Returns the session for this query.
std::shared_ptr<ImpalaServer::SessionState> session() const { return session_; }
/// Queries are run and authorized on behalf of the effective_user.
const std::string& effective_user() const;
@@ -244,23 +255,32 @@ class ClientRequestState {
/// Contents are only valid after InitExecRequest(TQueryCtx) initializes the
/// TExecRequest.
const TExecRequest& exec_request() const {
return exec_request_;
DCHECK(exec_request_ != nullptr);
return *exec_request_;
}
TStmtType::type stmt_type() const { return exec_request_.stmt_type; }
TStmtType::type stmt_type() const { return exec_request_->stmt_type; }
TCatalogOpType::type catalog_op_type() const {
return exec_request_.catalog_op_request.op_type;
return exec_request_->catalog_op_request.op_type;
}
TDdlType::type ddl_type() const {
return exec_request_.catalog_op_request.ddl_params.ddl_type;
return exec_request_->catalog_op_request.ddl_params.ddl_type;
}
std::mutex* lock() { return &lock_; }
std::mutex* fetch_rows_lock() { return &fetch_rows_lock_; }
/// ExecState is stored using an AtomicEnum, so reads do not require holding lock_.
ExecState exec_state() const { return exec_state_.Load(); }
/// Translate exec_state_ to a TOperationState.
/// RetryState is stored using an AtomicEnum, so reads do not require holding lock_.
RetryState retry_state() const { return retry_state_.Load(); }
/// Translate exec_state_ to a TOperationState. Returns the current TOperationState.
apache::hive::service::cli::thrift::TOperationState::type TOperationState() const;
/// Translate exec_state_ to a beeswax::QueryState.
/// Translate exec_state_ to a beeswax::QueryState. Returns the current
/// beeswax::QueryState.
beeswax::QueryState::type BeeswaxQueryState() const;
const Status& query_status() const { return query_status_; }
void set_result_metadata(const TResultSetMetadata& md) { result_metadata_ = md; }
void set_user_profile_access(bool user_has_profile_access) {
@@ -278,7 +298,7 @@ class ClientRequestState {
TUniqueId parent_query_id() const { return query_ctx_.parent_query_id; }
const std::vector<std::string>& GetAnalysisWarnings() const {
return exec_request_.analysis_warnings;
return exec_request_->analysis_warnings;
}
inline int64_t last_active_ms() const {
@@ -313,10 +333,56 @@ class ClientRequestState {
/// Returns the FETCH_ROWS_TIMEOUT_MS value for this query (converted to microseconds).
int64_t fetch_rows_timeout_us() const { return fetch_rows_timeout_us_; }
/// True if Finalize() was called.
bool started_finalize() const { return started_finalize_.Load(); }
/// Returns the max size of the result_cache_ in number of rows.
int64_t result_cache_max_size() const { return result_cache_max_size_; }
protected:
/// Sets the RetryState to RETRYING. Updates the runtime profile with the retry status
/// and cause. Must be called while 'lock_' is held. Sets the query_status_. Future
/// calls to UpdateQueryStatus will not have any effect. This is necessary to prevent
/// any future calls to UpdateQueryStatus from updating the ExecState to ERROR. The
/// ExecState should not be set to ERROR when a query is being retried in order to
/// prevent any error statuses from being exposed to the client.
void MarkAsRetrying(const Status& status);
/// Sets the RetryState to RETRIED and wakes up any threads waiting for the query to be
/// RETRIED. 'retried_id' is the query id of the newly retried query (e.g. not the id
/// of the "original" query that was submitted by the user, but the id of the new query
/// that was created because the "original" query failed and had to be retried).
void MarkAsRetried(const TUniqueId& retried_id);
/// Returns true if this ClientRequestState was created as a retry of a previously
/// failed query, false otherwise. This is different from WasRetried() which tracks
/// if this ClientRequestState was retried (retries are done in a new
/// ClientRequestState).
bool IsRetriedQuery() const { return original_id_ != nullptr; }
/// Only called if this is a "retried" query - e.g. it was created as a result of
/// retrying a failed query. It sets the 'original_id_' field, which is the query id of
/// the original query attempt that failed. The original query id is added to the
/// runtime profile as well.
void SetOriginalId(const TUniqueId& original_id);
/// Returns true if this ClientRequestState has already been retried, e.g. the
/// RetryState is either RETRYING or RETRIED.
bool WasRetried() const {
RetryState retry_state = retry_state_.Load();
return retry_state == RetryState::RETRYING || retry_state == RetryState::RETRIED;
}
/// Can only be called if this query is the result of retrying a previously failed
/// query. Returns the query id of the original query.
const TUniqueId& original_id() const {
DCHECK(original_id_ != nullptr);
return *original_id_;
}
/// Returns the QueryDriver that owns this ClientRequestState.
QueryDriver* parent_driver() const { return parent_driver_; }
/// Returns true if results cacheing is enabled, false otherwise.
bool IsResultCacheingEnabled() const { return result_cache_max_size_ >= 0; }
protected:
/// Updates the end_time_us_ of this query if it isn't set. The end time is determined
/// when this function is called for the first time, calling it multiple times does not
/// change the end time.
@@ -371,9 +437,6 @@ protected:
/// See "Locking" in the class comment for lock acquisition order.
std::mutex lock_;
/// TODO: remove and use ExecEnv::GetInstance() instead
ExecEnv* exec_env_;
/// Thread for asynchronously running Wait().
std::unique_ptr<Thread> wait_thread_;
@@ -480,14 +543,17 @@ protected:
/// UpdateExecState(), to ensure that the query profile is also updated.
AtomicEnum<ExecState> exec_state_{ExecState::INITIALIZED};
/// The current RetryState of the query.
AtomicEnum<RetryState> retry_state_{RetryState::NOT_RETRIED};
/// The current status of the query tracked by this ClientRequestState. Updated by
/// UpdateQueryStatus(Status).
/// UpdateQueryStatus(Status) or MarkAsRetrying(Status).
Status query_status_;
/// The TExecRequest for the query tracked by this ClientRequestState. The TExecRequest
/// is initialized in InitExecRequest(TQueryCtx). It should not be used until
/// InitExecRequest(TQueryCtx) has been called.
TExecRequest exec_request_;
/// is initialized in QueryDriver::RunFrontendPlanner(TQueryCtx).The TExecRequest is
/// owned by the parent QueryDriver.
TExecRequest* exec_request_;
/// If true, effective_user() has access to the runtime profile and execution
/// summary.
@@ -526,14 +592,24 @@ protected:
/// the coordinator releases its admission control resources.
AtomicInt64 end_time_us_{0};
/// True if a thread has called Finalize(). Threads calling Finalize()
/// do a compare-and-swap on this so that only one thread can proceed.
AtomicBool started_finalize_{false};
/// Timeout, in microseconds, when waiting for rows to become available. Derived from
/// the query option FETCH_ROWS_TIMEOUT_MS.
const int64_t fetch_rows_timeout_us_;
/// If this ClientRequestState was created as a retry of a previously failed query, the
/// original_id_ is set to the query id of the original query that failed. The
/// "original" query is the query that was submitted by the user that failed and had to
/// be retried.
std::unique_ptr<const TUniqueId> original_id_ = nullptr;
/// Condition variable used to signal any threads that are waiting until the query has
/// been retried.
ConditionVariable block_until_retried_cv_;
/// The QueryDriver that owns this ClientRequestState. The reference is set in the
/// constructor. It always outlives the ClientRequestState.
QueryDriver* parent_driver_;
/// Executes a local catalog operation (an operation that does not need to execute
/// against the catalog service). Includes USE, SHOW, DESCRIBE, and EXPLAIN statements.
Status ExecLocalCatalogOp(const TCatalogOpRequest& catalog_op) WARN_UNUSED_RESULT;
@@ -646,9 +722,6 @@ protected:
/// Logs audit and column lineage events. Expects that Wait() has already finished.
/// Grabs lock_ for polling the query_status(). Hence do not call it under lock_.
void LogQueryEvents();
/// Converts the given ExecState to a string representation.
std::string ExecStateToString(ExecState state) const;
};
}

View File

@@ -28,6 +28,7 @@
#include "runtime/exec-env.h"
#include "runtime/mem-tracker.h"
#include "runtime/query-exec-mgr.h"
#include "runtime/query-driver.h"
#include "runtime/query-state.h"
#include "service/client-request-state.h"
#include "service/impala-server.h"
@@ -165,13 +166,14 @@ void ControlService::ExecQueryFInstances(const ExecQueryFInstancesRequestPB* req
void ControlService::ReportExecStatus(const ReportExecStatusRequestPB* request,
ReportExecStatusResponsePB* response, RpcContext* rpc_context) {
const TUniqueId query_id = ProtoToQueryId(request->query_id());
shared_ptr<ClientRequestState> request_state =
ExecEnv::GetInstance()->impala_server()->GetClientRequestState(query_id);
QueryHandle query_handle;
Status status =
ExecEnv::GetInstance()->impala_server()->GetQueryHandle(query_id, &query_handle);
// This failpoint is to allow jitter to be injected.
DebugActionNoFail(FLAGS_debug_actions, "REPORT_EXEC_STATUS_DELAY");
if (request_state.get() == nullptr) {
if (!status.ok()) {
// This is expected occasionally (since a report RPC might be in flight while
// cancellation is happening). Return an error to the caller to get it to stop.
const string& err = Substitute("ReportExecStatus(): Received report for unknown "
@@ -190,10 +192,10 @@ void ControlService::ReportExecStatus(const ReportExecStatusRequestPB* request,
TRuntimeProfileForest thrift_profiles;
if (LIKELY(request->has_thrift_profiles_sidecar_idx())) {
const Status& profile_status =
GetProfile(*request, *request_state.get(), rpc_context, &thrift_profiles);
GetProfile(*request, *query_handle, rpc_context, &thrift_profiles);
if (UNLIKELY(!profile_status.ok())) {
LOG(ERROR) << Substitute("ReportExecStatus(): Failed to deserialize profile "
"for query ID $0: $1", PrintId(request_state->query_id()),
"for query ID $0: $1", PrintId(query_handle->query_id()),
profile_status.GetDetail());
// Do not expose a partially deserialized profile.
TRuntimeProfileForest empty_profiles;
@@ -201,7 +203,7 @@ void ControlService::ReportExecStatus(const ReportExecStatusRequestPB* request,
}
}
Status resp_status = request_state->UpdateBackendExecStatus(*request, thrift_profiles);
Status resp_status = query_handle->UpdateBackendExecStatus(*request, thrift_profiles);
RespondAndReleaseRpc(resp_status, response, rpc_context);
}

View File

@@ -24,6 +24,7 @@
#include "runtime/exec-env.h"
#include "runtime/raw-value.inline.h"
#include "runtime/timestamp-value.h"
#include "runtime/query-driver.h"
#include "service/client-request-state.h"
#include "service/frontend.h"
#include "service/query-options.h"
@@ -50,7 +51,7 @@ using namespace beeswax;
namespace impala {
void ImpalaServer::query(QueryHandle& query_handle, const Query& query) {
void ImpalaServer::query(beeswax::QueryHandle& beeswax_handle, const Query& query) {
VLOG_QUERY << "query(): query=" << query.query;
RAISE_IF_ERROR(CheckNotShuttingDown(), SQLSTATE_GENERAL_ERROR);
@@ -65,29 +66,29 @@ void ImpalaServer::query(QueryHandle& query_handle, const Query& query) {
// raise Syntax error or access violation; it's likely to be syntax/analysis error
// TODO: that may not be true; fix this
shared_ptr<ClientRequestState> request_state;
RAISE_IF_ERROR(Execute(&query_ctx, session, &request_state),
QueryHandle query_handle;
RAISE_IF_ERROR(Execute(&query_ctx, session, &query_handle),
SQLSTATE_SYNTAX_ERROR_OR_ACCESS_VIOLATION);
// start thread to wait for results to become available, which will allow
// us to advance query state to FINISHED or EXCEPTION
Status status = request_state->WaitAsync();
Status status = query_handle->WaitAsync();
if (!status.ok()) {
discard_result(UnregisterQuery(request_state->query_id(), false, &status));
discard_result(UnregisterQuery(query_handle->query_id(), false, &status));
RaiseBeeswaxException(status.GetDetail(), SQLSTATE_GENERAL_ERROR);
}
// Once the query is running do a final check for session closure and add it to the
// set of in-flight queries.
status = SetQueryInflight(session, request_state);
status = SetQueryInflight(session, query_handle);
if (!status.ok()) {
discard_result(UnregisterQuery(request_state->query_id(), false, &status));
discard_result(UnregisterQuery(query_handle->query_id(), false, &status));
RaiseBeeswaxException(status.GetDetail(), SQLSTATE_GENERAL_ERROR);
}
TUniqueIdToQueryHandle(request_state->query_id(), &query_handle);
TUniqueIdToBeeswaxHandle(query_handle->query_id(), &beeswax_handle);
}
void ImpalaServer::executeAndWait(QueryHandle& query_handle, const Query& query,
const LogContextId& client_ctx) {
void ImpalaServer::executeAndWait(beeswax::QueryHandle& beeswax_handle,
const Query& query, const LogContextId& client_ctx) {
VLOG_QUERY << "executeAndWait(): query=" << query.query;
RAISE_IF_ERROR(CheckNotShuttingDown(), SQLSTATE_GENERAL_ERROR);
ScopedSessionState session_handle(this);
@@ -99,7 +100,6 @@ void ImpalaServer::executeAndWait(QueryHandle& query_handle, const Query& query,
// raise general error for request conversion error;
RAISE_IF_ERROR(QueryToTQueryContext(query, &query_ctx), SQLSTATE_GENERAL_ERROR);
shared_ptr<ClientRequestState> request_state;
DCHECK(session != nullptr); // The session should exist.
{
// The session is created when the client connects. Depending on the underlying
@@ -111,32 +111,33 @@ void ImpalaServer::executeAndWait(QueryHandle& query_handle, const Query& query,
// raise Syntax error or access violation; it's likely to be syntax/analysis error
// TODO: that may not be true; fix this
RAISE_IF_ERROR(Execute(&query_ctx, session, &request_state),
QueryHandle query_handle;
RAISE_IF_ERROR(Execute(&query_ctx, session, &query_handle),
SQLSTATE_SYNTAX_ERROR_OR_ACCESS_VIOLATION);
// Once the query is running do a final check for session closure and add it to the
// set of in-flight queries.
Status status = SetQueryInflight(session, request_state);
Status status = SetQueryInflight(session, query_handle);
if (!status.ok()) {
discard_result(UnregisterQuery(request_state->query_id(), false, &status));
discard_result(UnregisterQuery(query_handle->query_id(), false, &status));
RaiseBeeswaxException(status.GetDetail(), SQLSTATE_GENERAL_ERROR);
}
// block until results are ready
request_state->Wait();
query_handle->Wait();
{
lock_guard<mutex> l(*request_state->lock());
status = request_state->query_status();
lock_guard<mutex> l(*query_handle->lock());
status = query_handle->query_status();
}
if (!status.ok()) {
discard_result(UnregisterQuery(request_state->query_id(), false, &status));
discard_result(UnregisterQuery(query_handle->query_id(), false, &status));
RaiseBeeswaxException(status.GetDetail(), SQLSTATE_GENERAL_ERROR);
}
TUniqueIdToQueryHandle(request_state->query_id(), &query_handle);
TUniqueIdToBeeswaxHandle(query_handle->query_id(), &beeswax_handle);
// If the input log context id is an empty string, then create a new number and
// set it to _return. Otherwise, set _return with the input log context
query_handle.log_context = client_ctx.empty() ? query_handle.id : client_ctx;
beeswax_handle.log_context = client_ctx.empty() ? beeswax_handle.id : client_ctx;
}
void ImpalaServer::explain(QueryExplanation& query_explanation, const Query& query) {
@@ -159,8 +160,9 @@ void ImpalaServer::explain(QueryExplanation& query_explanation, const Query& que
<< "\nplan: " << query_explanation.textual;
}
void ImpalaServer::fetch(Results& query_results, const QueryHandle& query_handle,
const bool start_over, const int32_t fetch_size) {
void ImpalaServer::fetch(Results& query_results,
const beeswax::QueryHandle& beeswax_handle, const bool start_over,
const int32_t fetch_size) {
ScopedSessionState session_handle(this);
shared_ptr<SessionState> session;
RAISE_IF_ERROR(session_handle.WithBeeswaxSession(
@@ -173,21 +175,18 @@ void ImpalaServer::fetch(Results& query_results, const QueryHandle& query_handle
}
TUniqueId query_id;
QueryHandleToTUniqueId(query_handle, &query_id);
BeeswaxHandleToTUniqueId(beeswax_handle, &query_id);
VLOG_ROW << "fetch(): query_id=" << PrintId(query_id) << " fetch_size=" << fetch_size;
shared_ptr<ClientRequestState> request_state = GetClientRequestState(query_id);
if (UNLIKELY(request_state == nullptr)) {
Status err = Status::Expected(TErrorCode::INVALID_QUERY_HANDLE, PrintId(query_id));
string err_msg = err.GetDetail();
VLOG(1) << err_msg;
RaiseBeeswaxException(err_msg, SQLSTATE_GENERAL_ERROR);
}
QueryHandle query_handle;
RAISE_IF_ERROR(GetActiveQueryHandle(query_id, &query_handle), SQLSTATE_GENERAL_ERROR);
// Validate that query can be accessed by user.
RAISE_IF_ERROR(CheckClientRequestSession(session.get(), request_state->effective_user(),
query_id), SQLSTATE_GENERAL_ERROR);
RAISE_IF_ERROR(CheckClientRequestSession(session.get(),
query_handle->effective_user(), query_id),
SQLSTATE_GENERAL_ERROR);
Status status =
FetchInternal(request_state.get(), start_over, fetch_size, &query_results);
FetchInternal(query_id, start_over, fetch_size, &query_results);
VLOG_ROW << "fetch result: #results=" << query_results.data.size()
<< " has_more=" << (query_results.has_more ? "true" : "false");
if (!status.ok()) {
@@ -198,7 +197,7 @@ void ImpalaServer::fetch(Results& query_results, const QueryHandle& query_handle
// TODO: Handle complex types.
void ImpalaServer::get_results_metadata(ResultsMetadata& results_metadata,
const QueryHandle& handle) {
const beeswax::QueryHandle& beeswax_handle) {
ScopedSessionState session_handle(this);
shared_ptr<SessionState> session;
RAISE_IF_ERROR(session_handle.WithBeeswaxSession(
@@ -206,22 +205,21 @@ void ImpalaServer::get_results_metadata(ResultsMetadata& results_metadata,
// Convert QueryHandle to TUniqueId and get the query exec state.
TUniqueId query_id;
QueryHandleToTUniqueId(handle, &query_id);
BeeswaxHandleToTUniqueId(beeswax_handle, &query_id);
VLOG_QUERY << "get_results_metadata(): query_id=" << PrintId(query_id);
shared_ptr<ClientRequestState> request_state = GetClientRequestState(query_id);
if (UNLIKELY(request_state.get() == nullptr)) {
Status err = Status::Expected(TErrorCode::INVALID_QUERY_HANDLE, PrintId(query_id));
RaiseBeeswaxException(err.GetDetail(), SQLSTATE_GENERAL_ERROR);
}
QueryHandle query_handle;
RAISE_IF_ERROR(GetActiveQueryHandle(query_id, &query_handle), SQLSTATE_GENERAL_ERROR);
// Validate that query can be accessed by user.
RAISE_IF_ERROR(CheckClientRequestSession(session.get(), request_state->effective_user(),
RAISE_IF_ERROR(CheckClientRequestSession(session.get(), query_handle->effective_user(),
query_id), SQLSTATE_GENERAL_ERROR);
{
lock_guard<mutex> l(*request_state->lock());
lock_guard<mutex> l(*query_handle->lock());
// Convert TResultSetMetadata to Beeswax.ResultsMetadata
const TResultSetMetadata* result_set_md = request_state->result_metadata();
const TResultSetMetadata* result_set_md = query_handle->result_metadata();
results_metadata.__isset.schema = true;
results_metadata.schema.__isset.fieldSchemas = true;
results_metadata.schema.fieldSchemas.resize(result_set_md->columns.size());
@@ -246,12 +244,12 @@ void ImpalaServer::get_results_metadata(ResultsMetadata& results_metadata,
// results_metadata.table_dir and in_tablename are not applicable.
}
void ImpalaServer::close(const QueryHandle& handle) {
void ImpalaServer::close(const beeswax::QueryHandle& beeswax_handle) {
ScopedSessionState session_handle(this);
RAISE_IF_ERROR(session_handle.WithBeeswaxSession(ThriftServer::GetThreadConnectionId()),
SQLSTATE_GENERAL_ERROR);
TUniqueId query_id;
QueryHandleToTUniqueId(handle, &query_id);
BeeswaxHandleToTUniqueId(beeswax_handle, &query_id);
// Impala-shell and administrative tools can call this from a different connection,
// e.g. to allow an admin to force-terminate queries. We should allow the operation to
@@ -261,34 +259,34 @@ void ImpalaServer::close(const QueryHandle& handle) {
VLOG_QUERY << "close(): query_id=" << PrintId(query_id);
// TODO: do we need to raise an exception if the query state is EXCEPTION?
// TODO: use timeout to get rid of unwanted request_state.
// TODO: use timeout to get rid of unwanted query_handle.
RAISE_IF_ERROR(UnregisterQuery(query_id, true), SQLSTATE_GENERAL_ERROR);
}
beeswax::QueryState::type ImpalaServer::get_state(const QueryHandle& handle) {
beeswax::QueryState::type ImpalaServer::get_state(
const beeswax::QueryHandle& beeswax_handle) {
ScopedSessionState session_handle(this);
shared_ptr<SessionState> session;
RAISE_IF_ERROR(session_handle.WithBeeswaxSession(
ThriftServer::GetThreadConnectionId(), &session), SQLSTATE_GENERAL_ERROR);
TUniqueId query_id;
QueryHandleToTUniqueId(handle, &query_id);
BeeswaxHandleToTUniqueId(beeswax_handle, &query_id);
VLOG_ROW << "get_state(): query_id=" << PrintId(query_id);
shared_ptr<ClientRequestState> request_state = GetClientRequestState(query_id);
if (UNLIKELY(request_state == nullptr)) {
VLOG_QUERY << "ImpalaServer::get_state invalid handle";
Status err = Status::Expected(TErrorCode::INVALID_QUERY_HANDLE, PrintId(query_id));
RaiseBeeswaxException(err.GetDetail(), SQLSTATE_GENERAL_ERROR);
}
QueryHandle query_handle;
RAISE_IF_ERROR(GetActiveQueryHandle(query_id, &query_handle), SQLSTATE_GENERAL_ERROR);
// Validate that query can be accessed by user.
RAISE_IF_ERROR(CheckClientRequestSession(session.get(), request_state->effective_user(),
RAISE_IF_ERROR(CheckClientRequestSession(session.get(), query_handle->effective_user(),
query_id), SQLSTATE_GENERAL_ERROR);
// Take the lock to ensure that if the client sees a query_state == EXCEPTION, it is
// guaranteed to see the error query_status.
lock_guard<mutex> l(*request_state->lock());
beeswax::QueryState::type query_state = request_state->BeeswaxQueryState();
DCHECK_EQ(query_state == beeswax::QueryState::EXCEPTION,
!request_state->query_status().ok());
lock_guard<mutex> l(*query_handle->lock());
beeswax::QueryState::type query_state = query_handle->BeeswaxQueryState();
DCHECK_EQ(query_state == beeswax::QueryState::EXCEPTION
|| query_handle->retry_state() == ClientRequestState::RetryState::RETRYING
|| query_handle->retry_state() == ClientRequestState::RetryState::RETRIED,
!query_handle->query_status().ok());
return query_state;
}
@@ -308,43 +306,40 @@ void ImpalaServer::get_log(string& log, const LogContextId& context) {
RAISE_IF_ERROR(session_handle.WithBeeswaxSession(
ThriftServer::GetThreadConnectionId(), &session), SQLSTATE_GENERAL_ERROR);
// LogContextId is the same as QueryHandle.id
QueryHandle handle;
handle.__set_id(context);
beeswax::QueryHandle beeswax_handle;
beeswax_handle.__set_id(context);
TUniqueId query_id;
QueryHandleToTUniqueId(handle, &query_id);
BeeswaxHandleToTUniqueId(beeswax_handle, &query_id);
QueryHandle query_handle;
RAISE_IF_ERROR(GetActiveQueryHandle(query_id, &query_handle), SQLSTATE_GENERAL_ERROR);
shared_ptr<ClientRequestState> request_state = GetClientRequestState(query_id);
if (request_state.get() == nullptr) {
stringstream str;
str << "unknown query id: " << PrintId(query_id);
LOG(ERROR) << str.str();
return;
}
// Validate that query can be accessed by user.
RAISE_IF_ERROR(CheckClientRequestSession(session.get(), request_state->effective_user(),
query_id), SQLSTATE_GENERAL_ERROR);
RAISE_IF_ERROR(
CheckClientRequestSession(session.get(), query_handle->effective_user(), query_id),
SQLSTATE_GENERAL_ERROR);
stringstream error_log_ss;
{
// Take the lock to ensure that if the client sees a exec_state == ERROR, it is
// guaranteed to see the error query_status.
lock_guard<mutex> l(*request_state->lock());
DCHECK_EQ(request_state->exec_state() == ClientRequestState::ExecState::ERROR,
!request_state->query_status().ok());
lock_guard<mutex> l(*query_handle->lock());
DCHECK_EQ(query_handle->exec_state() == ClientRequestState::ExecState::ERROR,
!query_handle->query_status().ok());
// If the query status is !ok, include the status error message at the top of the log.
if (!request_state->query_status().ok()) {
error_log_ss << request_state->query_status().GetDetail() << "\n";
if (!query_handle->query_status().ok()) {
error_log_ss << query_handle->query_status().GetDetail() << "\n";
}
}
// Add warnings from analysis
for (const string& warning : request_state->GetAnalysisWarnings()) {
for (const string& warning : query_handle->GetAnalysisWarnings()) {
error_log_ss << warning << "\n";
}
// Add warnings from execution
if (request_state->GetCoordinator() != nullptr) {
const std::string coord_errors = request_state->GetCoordinator()->GetErrorLog();
if (query_handle->GetCoordinator() != nullptr) {
const std::string coord_errors = query_handle->GetCoordinator()->GetErrorLog();
if (!coord_errors.empty()) error_log_ss << coord_errors << "\n";
}
log = error_log_ss.str();
@@ -367,13 +362,13 @@ void ImpalaServer::dump_config(string& config) {
}
void ImpalaServer::Cancel(impala::TStatus& tstatus,
const beeswax::QueryHandle& query_handle) {
const beeswax::QueryHandle& beeswax_handle) {
ScopedSessionState session_handle(this);
RAISE_IF_ERROR(session_handle.WithBeeswaxSession(ThriftServer::GetThreadConnectionId()),
SQLSTATE_GENERAL_ERROR);
// Convert QueryHandle to TUniqueId and get the query exec state.
TUniqueId query_id;
QueryHandleToTUniqueId(query_handle, &query_id);
BeeswaxHandleToTUniqueId(beeswax_handle, &query_id);
// Impala-shell and administrative tools can call this from a different connection,
// e.g. to allow an admin to force-terminate queries. We should allow the operation to
@@ -385,13 +380,13 @@ void ImpalaServer::Cancel(impala::TStatus& tstatus,
}
void ImpalaServer::CloseInsert(TDmlResult& dml_result,
const QueryHandle& query_handle) {
const beeswax::QueryHandle& beeswax_handle) {
ScopedSessionState session_handle(this);
shared_ptr<SessionState> session;
RAISE_IF_ERROR(session_handle.WithBeeswaxSession(
ThriftServer::GetThreadConnectionId(), &session), SQLSTATE_GENERAL_ERROR);
TUniqueId query_id;
QueryHandleToTUniqueId(query_handle, &query_id);
BeeswaxHandleToTUniqueId(beeswax_handle, &query_id);
VLOG_QUERY << "CloseInsert(): query_id=" << PrintId(query_id);
// CloseInsertInternal() will validates that 'session' has access to 'query_id'.
@@ -404,7 +399,8 @@ void ImpalaServer::CloseInsert(TDmlResult& dml_result,
// Gets the runtime profile string for the given query handle and stores the result in
// the profile_output parameter. Raises a BeeswaxException if there are any errors
// getting the profile, such as no matching queries found.
void ImpalaServer::GetRuntimeProfile(string& profile_output, const QueryHandle& handle) {
void ImpalaServer::GetRuntimeProfile(
string& profile_output, const beeswax::QueryHandle& beeswax_handle) {
ScopedSessionState session_handle(this);
const TUniqueId& session_id = ThriftServer::GetThreadConnectionId();
stringstream ss;
@@ -416,12 +412,21 @@ void ImpalaServer::GetRuntimeProfile(string& profile_output, const QueryHandle&
RaiseBeeswaxException(ss.str(), SQLSTATE_GENERAL_ERROR);
}
TUniqueId query_id;
QueryHandleToTUniqueId(handle, &query_id);
// GetRuntimeProfile() will validate that the user has access to 'query_id'.
BeeswaxHandleToTUniqueId(beeswax_handle, &query_id);
VLOG_RPC << "GetRuntimeProfile(): query_id=" << PrintId(query_id);
Status status = GetRuntimeProfileOutput(
query_id, GetEffectiveUser(*session), TRuntimeProfileFormat::STRING,
&ss, nullptr, nullptr);
// If the query was retried, fetch the profile for the most recent attempt of the query
// The original query profile should still be accessible via the web ui.
QueryHandle query_handle;
Status status = GetActiveQueryHandle(query_id, &query_handle);
if (LIKELY(status.ok())) {
query_id = query_handle->query_id();
}
// GetRuntimeProfile() will validate that the user has access to 'query_id'.
status = GetRuntimeProfileOutput(query_id, GetEffectiveUser(*session),
TRuntimeProfileFormat::STRING, &ss, nullptr, nullptr);
if (!status.ok()) {
ss << "GetRuntimeProfile error: " << status.GetDetail();
RaiseBeeswaxException(ss.str(), SQLSTATE_GENERAL_ERROR);
@@ -430,7 +435,7 @@ void ImpalaServer::GetRuntimeProfile(string& profile_output, const QueryHandle&
}
void ImpalaServer::GetExecSummary(impala::TExecSummary& result,
const beeswax::QueryHandle& handle) {
const beeswax::QueryHandle& beeswax_handle) {
ScopedSessionState session_handle(this);
const TUniqueId& session_id = ThriftServer::GetThreadConnectionId();
shared_ptr<SessionState> session;
@@ -442,7 +447,7 @@ void ImpalaServer::GetExecSummary(impala::TExecSummary& result,
RaiseBeeswaxException(ss.str(), SQLSTATE_GENERAL_ERROR);
}
TUniqueId query_id;
QueryHandleToTUniqueId(handle, &query_id);
BeeswaxHandleToTUniqueId(beeswax_handle, &query_id);
VLOG_RPC << "GetExecSummary(): query_id=" << PrintId(query_id);
// GetExecSummary() will validate that the user has access to 'query_id'.
Status status = GetExecSummary(query_id, GetEffectiveUser(*session), &result);
@@ -513,16 +518,16 @@ Status ImpalaServer::QueryToTQueryContext(const Query& query,
return Status::OK();
}
inline void ImpalaServer::TUniqueIdToQueryHandle(const TUniqueId& query_id,
QueryHandle* handle) {
inline void ImpalaServer::TUniqueIdToBeeswaxHandle(
const TUniqueId& query_id, beeswax::QueryHandle* beeswax_handle) {
string query_id_str = PrintId(query_id);
handle->__set_id(query_id_str);
handle->__set_log_context(query_id_str);
beeswax_handle->__set_id(query_id_str);
beeswax_handle->__set_log_context(query_id_str);
}
inline void ImpalaServer::QueryHandleToTUniqueId(const QueryHandle& handle,
TUniqueId* query_id) {
ParseId(handle.id, query_id);
inline void ImpalaServer::BeeswaxHandleToTUniqueId(
const beeswax::QueryHandle& beeswax_handle, TUniqueId* query_id) {
ParseId(beeswax_handle.id, query_id);
}
[[noreturn]] void ImpalaServer::RaiseBeeswaxException(
@@ -533,15 +538,14 @@ inline void ImpalaServer::QueryHandleToTUniqueId(const QueryHandle& handle,
throw exc;
}
Status ImpalaServer::FetchInternal(ClientRequestState* request_state,
const bool start_over, const int32_t fetch_size, beeswax::Results* query_results) {
// Make sure ClientRequestState::Wait() has completed before fetching rows. Wait()
// ensures that rows are ready to be fetched (e.g., Wait() opens
// ClientRequestState::output_exprs_, which are evaluated in
// ClientRequestState::FetchRows() below).
Status ImpalaServer::FetchInternal(TUniqueId query_id, const bool start_over,
const int32_t fetch_size, beeswax::Results* query_results) {
bool timed_out = false;
int64_t block_on_wait_time_us = 0;
if (!request_state->BlockOnWait(
request_state->fetch_rows_timeout_us(), &block_on_wait_time_us)) {
QueryHandle query_handle;
RETURN_IF_ERROR(
WaitForResults(query_id, &query_handle, &block_on_wait_time_us, &timed_out));
if (timed_out) {
query_results->__set_ready(false);
query_results->__set_has_more(true);
query_results->__isset.columns = false;
@@ -549,19 +553,19 @@ Status ImpalaServer::FetchInternal(ClientRequestState* request_state,
return Status::OK();
}
lock_guard<mutex> frl(*request_state->fetch_rows_lock());
lock_guard<mutex> l(*request_state->lock());
lock_guard<mutex> frl(*query_handle->fetch_rows_lock());
lock_guard<mutex> l(*query_handle->lock());
if (request_state->num_rows_fetched() == 0) {
request_state->set_fetched_rows();
if (query_handle->num_rows_fetched() == 0) {
query_handle->set_fetched_rows();
}
// Check for cancellation or an error.
RETURN_IF_ERROR(request_state->query_status());
RETURN_IF_ERROR(query_handle->query_status());
// ODBC-190: set Beeswax's Results.columns to work around bug ODBC-190;
// TODO: remove the block of code when ODBC-190 is resolved.
const TResultSetMetadata* result_metadata = request_state->result_metadata();
const TResultSetMetadata* result_metadata = query_handle->result_metadata();
query_results->columns.resize(result_metadata->columns.size());
for (int i = 0; i < result_metadata->columns.size(); ++i) {
// TODO: As of today, the ODBC driver does not support boolean and timestamp data
@@ -581,38 +585,32 @@ Status ImpalaServer::FetchInternal(ClientRequestState* request_state,
query_results->__set_ready(true);
// It's likely that ODBC doesn't care about start_row, but Hue needs it. For Hue,
// start_row starts from zero, not one.
query_results->__set_start_row(request_state->num_rows_fetched());
query_results->__set_start_row(query_handle->num_rows_fetched());
Status fetch_rows_status;
query_results->data.clear();
if (!request_state->eos()) {
if (!query_handle->eos()) {
scoped_ptr<QueryResultSet> result_set(QueryResultSet::CreateAsciiQueryResultSet(
*request_state->result_metadata(), &query_results->data));
*query_handle->result_metadata(), &query_results->data));
fetch_rows_status =
request_state->FetchRows(fetch_size, result_set.get(), block_on_wait_time_us);
query_handle->FetchRows(fetch_size, result_set.get(), block_on_wait_time_us);
}
query_results->__set_has_more(!request_state->eos());
query_results->__set_has_more(!query_handle->eos());
query_results->__isset.data = true;
return fetch_rows_status;
}
Status ImpalaServer::CloseInsertInternal(SessionState* session, const TUniqueId& query_id,
TDmlResult* dml_result) {
shared_ptr<ClientRequestState> request_state = GetClientRequestState(query_id);
if (UNLIKELY(request_state == nullptr)) {
Status err = Status::Expected(TErrorCode::INVALID_QUERY_HANDLE, PrintId(query_id));
VLOG(1) << err.GetDetail();
return err;
}
Status ImpalaServer::CloseInsertInternal(
SessionState* session, const TUniqueId& query_id, TDmlResult* dml_result) {
QueryHandle query_handle;
RAISE_IF_ERROR(GetActiveQueryHandle(query_id, &query_handle), SQLSTATE_GENERAL_ERROR);
RETURN_IF_ERROR(
CheckClientRequestSession(session, request_state->effective_user(), query_id));
CheckClientRequestSession(session, query_handle->effective_user(), query_id));
Status query_status;
request_state->GetDmlStats(dml_result, &query_status);
query_handle->GetDmlStats(dml_result, &query_status);
RETURN_IF_ERROR(UnregisterQuery(query_id, true));
return query_status;
}
}

View File

@@ -42,6 +42,7 @@
#include "runtime/coordinator.h"
#include "runtime/exec-env.h"
#include "runtime/raw-value.h"
#include "runtime/query-driver.h"
#include "scheduling/admission-controller.h"
#include "service/client-request-state.h"
#include "service/hs2-util.h"
@@ -136,7 +137,6 @@ void ImpalaServer::ExecuteMetadataOp(const THandleIdentifier& session_handle,
session->ToThrift(session_id, &query_ctx.session);
request->__set_session(query_ctx.session);
shared_ptr<ClientRequestState> request_state;
// There is no user-supplied query text available because this metadata operation comes
// from an RPC. As a best effort, we use the type of the operation.
map<int, const char*>::const_iterator query_text_it =
@@ -144,9 +144,9 @@ void ImpalaServer::ExecuteMetadataOp(const THandleIdentifier& session_handle,
const string& query_text = query_text_it == _TMetadataOpcode_VALUES_TO_NAMES.end() ?
"N/A" : query_text_it->second;
query_ctx.client_request.stmt = query_text;
request_state.reset(new ClientRequestState(query_ctx, exec_env_,
exec_env_->frontend(), this, session));
Status register_status = RegisterQuery(session, request_state);
QueryHandle query_handle;
QueryDriver::CreateNewDriver(this, &query_handle, query_ctx, session);
Status register_status = RegisterQuery(query_ctx.query_id, session, &query_handle);
if (!register_status.ok()) {
status->__set_statusCode(thrift::TStatusCode::ERROR_STATUS);
status->__set_errorMessage(register_status.GetDetail());
@@ -154,20 +154,20 @@ void ImpalaServer::ExecuteMetadataOp(const THandleIdentifier& session_handle,
return;
}
Status exec_status = request_state->Exec(*request);
Status exec_status = query_handle->Exec(*request);
if (!exec_status.ok()) {
discard_result(UnregisterQuery(request_state->query_id(), false, &exec_status));
discard_result(UnregisterQuery(query_handle->query_id(), false, &exec_status));
status->__set_statusCode(thrift::TStatusCode::ERROR_STATUS);
status->__set_errorMessage(exec_status.GetDetail());
status->__set_sqlState(SQLSTATE_GENERAL_ERROR);
return;
}
request_state->UpdateNonErrorExecState(ClientRequestState::ExecState::FINISHED);
query_handle->UpdateNonErrorExecState(ClientRequestState::ExecState::FINISHED);
Status inflight_status = SetQueryInflight(session, request_state);
Status inflight_status = SetQueryInflight(session, query_handle);
if (!inflight_status.ok()) {
discard_result(UnregisterQuery(request_state->query_id(), false, &inflight_status));
discard_result(UnregisterQuery(query_handle->query_id(), false, &inflight_status));
status->__set_statusCode(thrift::TStatusCode::ERROR_STATUS);
status->__set_errorMessage(inflight_status.GetDetail());
status->__set_sqlState(SQLSTATE_GENERAL_ERROR);
@@ -175,53 +175,52 @@ void ImpalaServer::ExecuteMetadataOp(const THandleIdentifier& session_handle,
}
handle->__set_hasResultSet(true);
// Secret is inherited from session.
TUniqueId operation_id = request_state->query_id();
TUniqueId operation_id = query_handle->query_id();
TUniqueIdToTHandleIdentifier(operation_id, secret, &(handle->operationId));
status->__set_statusCode(thrift::TStatusCode::SUCCESS_STATUS);
}
Status ImpalaServer::FetchInternal(ClientRequestState* request_state,
SessionState* session, int32_t fetch_size, bool fetch_first,
TFetchResultsResp* fetch_results, int32_t* num_results) {
// Make sure ClientRequestState::Wait() has completed before fetching rows. Wait()
// ensures that rows are ready to be fetched (e.g., Wait() opens
// ClientRequestState::output_exprs_, which are evaluated in
// ClientRequestState::FetchRows() below).
Status ImpalaServer::FetchInternal(TUniqueId query_id, SessionState* session,
int32_t fetch_size, bool fetch_first, TFetchResultsResp* fetch_results,
int32_t* num_results) {
bool timed_out = false;
int64_t block_on_wait_time_us = 0;
if (!request_state->BlockOnWait(
request_state->fetch_rows_timeout_us(), &block_on_wait_time_us)) {
QueryHandle query_handle;
RETURN_IF_ERROR(
WaitForResults(query_id, &query_handle, &block_on_wait_time_us, &timed_out));
if (timed_out) {
fetch_results->status.__set_statusCode(thrift::TStatusCode::STILL_EXECUTING_STATUS);
fetch_results->__set_hasMoreRows(true);
fetch_results->__isset.results = false;
return Status::OK();
}
lock_guard<mutex> frl(*request_state->fetch_rows_lock());
lock_guard<mutex> l(*request_state->lock());
lock_guard<mutex> frl(*query_handle->fetch_rows_lock());
lock_guard<mutex> l(*query_handle->lock());
// Check for cancellation or an error.
RETURN_IF_ERROR(request_state->query_status());
RETURN_IF_ERROR(query_handle->query_status());
if (request_state->num_rows_fetched() == 0) {
request_state->set_fetched_rows();
if (query_handle->num_rows_fetched() == 0) {
query_handle->set_fetched_rows();
}
if (fetch_first) RETURN_IF_ERROR(request_state->RestartFetch());
if (fetch_first) RETURN_IF_ERROR(query_handle->RestartFetch());
fetch_results->results.__set_startRowOffset(request_state->num_rows_fetched());
fetch_results->results.__set_startRowOffset(query_handle->num_rows_fetched());
// Child queries should always return their results in row-major format, rather than
// inheriting the parent session's setting.
bool is_child_query = request_state->parent_query_id() != TUniqueId();
bool is_child_query = query_handle->parent_query_id() != TUniqueId();
TProtocolVersion::type version = is_child_query ?
TProtocolVersion::HIVE_CLI_SERVICE_PROTOCOL_V1 : session->hs2_version;
scoped_ptr<QueryResultSet> result_set(QueryResultSet::CreateHS2ResultSet(
version, *(request_state->result_metadata()), &(fetch_results->results)));
version, *(query_handle->result_metadata()), &(fetch_results->results)));
RETURN_IF_ERROR(
request_state->FetchRows(fetch_size, result_set.get(), block_on_wait_time_us));
query_handle->FetchRows(fetch_size, result_set.get(), block_on_wait_time_us));
*num_results = result_set->size();
fetch_results->__isset.results = true;
fetch_results->__set_hasMoreRows(!request_state->eos());
fetch_results->__set_hasMoreRows(!query_handle->eos());
return Status::OK();
}
@@ -482,31 +481,27 @@ void ImpalaServer::ExecuteStatement(TExecuteStatementResp& return_val,
}
}
shared_ptr<ClientRequestState> request_state;
status = Execute(&query_ctx, session, &request_state);
QueryHandle query_handle;
status = Execute(&query_ctx, session, &query_handle);
HS2_RETURN_IF_ERROR(return_val, status, SQLSTATE_GENERAL_ERROR);
// Start thread to wait for results to become available.
status = request_state->WaitAsync();
status = query_handle->WaitAsync();
if (!status.ok()) goto return_error;
// Optionally enable result caching on the ClientRequestState.
if (cache_num_rows > 0) {
status = request_state->SetResultCache(
QueryResultSet::CreateHS2ResultSet(
session->hs2_version, *request_state->result_metadata(), nullptr),
cache_num_rows);
if (!status.ok()) goto return_error;
}
status = SetupResultsCacheing(query_handle, session, cache_num_rows);
if (!status.ok()) goto return_error;
// Once the query is running do a final check for session closure and add it to the
// set of in-flight queries.
status = SetQueryInflight(session, request_state);
status = SetQueryInflight(session, query_handle);
if (!status.ok()) goto return_error;
return_val.__isset.operationHandle = true;
return_val.operationHandle.__set_operationType(TOperationType::EXECUTE_STATEMENT);
return_val.operationHandle.__set_hasResultSet(request_state->returns_result_set());
return_val.operationHandle.__set_hasResultSet(query_handle->returns_result_set());
// Secret is inherited from session.
TUniqueIdToTHandleIdentifier(request_state->query_id(), secret,
TUniqueIdToTHandleIdentifier(query_handle->query_id(), secret,
&return_val.operationHandle.operationId);
return_val.status.__set_statusCode(
apache::hive::service::cli::thrift::TStatusCode::SUCCESS_STATUS);
@@ -515,10 +510,22 @@ void ImpalaServer::ExecuteStatement(TExecuteStatementResp& return_val,
return;
return_error:
discard_result(UnregisterQuery(request_state->query_id(), false, &status));
discard_result(UnregisterQuery(query_handle->query_id(), false, &status));
HS2_RETURN_ERROR(return_val, status.GetDetail(), SQLSTATE_GENERAL_ERROR);
}
Status ImpalaServer::SetupResultsCacheing(const QueryHandle& query_handle,
shared_ptr<SessionState> session, int64_t cache_num_rows) {
// Optionally enable result caching on the ClientRequestState.
if (cache_num_rows > 0) {
const TResultSetMetadata* result_set_md = query_handle->result_metadata();
QueryResultSet* result_set =
QueryResultSet::CreateHS2ResultSet(session->hs2_version, *result_set_md, nullptr);
RETURN_IF_ERROR(query_handle->SetResultCache(result_set, cache_num_rows));
}
return Status::OK();
}
void ImpalaServer::GetTypeInfo(TGetTypeInfoResp& return_val,
const TGetTypeInfoReq& request) {
VLOG_QUERY << "GetTypeInfo(): request=" << ThriftDebugString(request);
@@ -708,15 +715,12 @@ void ImpalaServer::GetOperationStatus(TGetOperationStatusResp& return_val,
SQLSTATE_GENERAL_ERROR);
VLOG_ROW << "GetOperationStatus(): query_id=" << PrintId(query_id);
shared_ptr<ClientRequestState> request_state = GetClientRequestState(query_id);
if (UNLIKELY(request_state.get() == nullptr)) {
Status err = Status::Expected(TErrorCode::INVALID_QUERY_HANDLE, PrintId(query_id));
string err_msg = err.GetDetail();
HS2_RETURN_ERROR(return_val, err_msg, SQLSTATE_GENERAL_ERROR);
}
QueryHandle query_handle;
HS2_RETURN_IF_ERROR(
return_val, GetActiveQueryHandle(query_id, &query_handle), SQLSTATE_GENERAL_ERROR);
ScopedSessionState session_handle(this);
const TUniqueId session_id = request_state->session_id();
const TUniqueId session_id = query_handle->session_id();
shared_ptr<SessionState> session;
HS2_RETURN_IF_ERROR(return_val,
session_handle.WithSession(
@@ -724,15 +728,19 @@ void ImpalaServer::GetOperationStatus(TGetOperationStatusResp& return_val,
SQLSTATE_GENERAL_ERROR);
{
lock_guard<mutex> l(*request_state->lock());
TOperationState::type operation_state = request_state->TOperationState();
lock_guard<mutex> l(*query_handle->lock());
TOperationState::type operation_state = query_handle->TOperationState();
return_val.__set_operationState(operation_state);
if (operation_state == TOperationState::ERROR_STATE) {
DCHECK(!request_state->query_status().ok());
return_val.__set_errorMessage(request_state->query_status().GetDetail());
DCHECK(!query_handle->query_status().ok());
return_val.__set_errorMessage(query_handle->query_status().GetDetail());
return_val.__set_sqlState(SQLSTATE_GENERAL_ERROR);
} else {
DCHECK(request_state->query_status().ok());
ClientRequestState::RetryState retry_state = query_handle->retry_state();
if (retry_state != ClientRequestState::RetryState::RETRYING
&& retry_state != ClientRequestState::RetryState::RETRIED) {
DCHECK(query_handle->query_status().ok());
}
}
}
}
@@ -746,14 +754,12 @@ void ImpalaServer::CancelOperation(TCancelOperationResp& return_val,
SQLSTATE_GENERAL_ERROR);
VLOG_QUERY << "CancelOperation(): query_id=" << PrintId(query_id);
shared_ptr<ClientRequestState> request_state = GetClientRequestState(query_id);
if (UNLIKELY(request_state.get() == nullptr)) {
Status err = Status::Expected(TErrorCode::INVALID_QUERY_HANDLE, PrintId(query_id));
string err_msg = err.GetDetail();
HS2_RETURN_ERROR(return_val, err_msg, SQLSTATE_GENERAL_ERROR);
}
QueryHandle query_handle;
HS2_RETURN_IF_ERROR(
return_val, GetActiveQueryHandle(query_id, &query_handle), SQLSTATE_GENERAL_ERROR);
ScopedSessionState session_handle(this);
const TUniqueId session_id = request_state->session_id();
const TUniqueId session_id = query_handle->session_id();
HS2_RETURN_IF_ERROR(return_val,
session_handle.WithSession(session_id, SecretArg::Operation(op_secret, query_id)),
SQLSTATE_GENERAL_ERROR);
@@ -779,24 +785,23 @@ void ImpalaServer::CloseImpalaOperation(TCloseImpalaOperationResp& return_val,
SQLSTATE_GENERAL_ERROR);
VLOG_QUERY << "CloseOperation(): query_id=" << PrintId(query_id);
shared_ptr<ClientRequestState> request_state = GetClientRequestState(query_id);
if (UNLIKELY(request_state.get() == nullptr)) {
Status err = Status::Expected(TErrorCode::INVALID_QUERY_HANDLE, PrintId(query_id));
HS2_RETURN_ERROR(return_val, err.GetDetail(), SQLSTATE_GENERAL_ERROR);
}
QueryHandle query_handle;
HS2_RETURN_IF_ERROR(
return_val, GetActiveQueryHandle(query_id, &query_handle), SQLSTATE_GENERAL_ERROR);
ScopedSessionState session_handle(this);
const TUniqueId session_id = request_state->session_id();
const TUniqueId session_id = query_handle->session_id();
HS2_RETURN_IF_ERROR(return_val,
session_handle.WithSession(session_id, SecretArg::Operation(op_secret, query_id)),
SQLSTATE_GENERAL_ERROR);
if (request_state->stmt_type() == TStmtType::DML) {
if (query_handle->stmt_type() == TStmtType::DML) {
Status query_status;
if (request_state->GetDmlStats(&return_val.dml_result, &query_status)) {
if (query_handle->GetDmlStats(&return_val.dml_result, &query_status)) {
return_val.__isset.dml_result = true;
}
}
// TODO: use timeout to get rid of unwanted request_state.
// TODO: use timeout to get rid of unwanted query_handle.
HS2_RETURN_IF_ERROR(return_val, UnregisterQuery(query_id, true),
SQLSTATE_GENERAL_ERROR);
return_val.status.__set_statusCode(thrift::TStatusCode::SUCCESS_STATUS);
@@ -812,22 +817,20 @@ void ImpalaServer::GetResultSetMetadata(TGetResultSetMetadataResp& return_val,
SQLSTATE_GENERAL_ERROR);
VLOG_QUERY << "GetResultSetMetadata(): query_id=" << PrintId(query_id);
shared_ptr<ClientRequestState> request_state = GetClientRequestState(query_id);
if (UNLIKELY(request_state.get() == nullptr)) {
VLOG_QUERY << "GetResultSetMetadata(): invalid query handle";
Status err = Status::Expected(TErrorCode::INVALID_QUERY_HANDLE, PrintId(query_id));
HS2_RETURN_ERROR(return_val, err.GetDetail(), SQLSTATE_GENERAL_ERROR);
}
QueryHandle query_handle;
HS2_RETURN_IF_ERROR(
return_val, GetActiveQueryHandle(query_id, &query_handle), SQLSTATE_GENERAL_ERROR);
ScopedSessionState session_handle(this);
const TUniqueId session_id = request_state->session_id();
const TUniqueId session_id = query_handle->session_id();
HS2_RETURN_IF_ERROR(return_val,
session_handle.WithSession(session_id, SecretArg::Operation(op_secret, query_id)),
SQLSTATE_GENERAL_ERROR);
{
lock_guard<mutex> l(*request_state->lock());
lock_guard<mutex> l(*query_handle->lock());
// Convert TResultSetMetadata to TGetResultSetMetadataResp
const TResultSetMetadata* result_set_md = request_state->result_metadata();
const TResultSetMetadata* result_set_md = query_handle->result_metadata();
DCHECK(result_set_md != NULL);
if (result_set_md->columns.size() > 0) {
return_val.__isset.schema = true;
@@ -865,17 +868,14 @@ void ImpalaServer::FetchResults(TFetchResultsResp& return_val,
VLOG_ROW << "FetchResults(): query_id=" << PrintId(query_id)
<< " fetch_size=" << request.maxRows;
shared_ptr<ClientRequestState> request_state = GetClientRequestState(query_id);
if (UNLIKELY(request_state == nullptr)) {
Status err = Status::Expected(TErrorCode::INVALID_QUERY_HANDLE, PrintId(query_id));
string err_msg = err.GetDetail();
VLOG(1) << err_msg;
HS2_RETURN_ERROR(return_val, err_msg, SQLSTATE_GENERAL_ERROR);
}
QueryHandle query_handle;
HS2_RETURN_IF_ERROR(
return_val, GetActiveQueryHandle(query_id, &query_handle), SQLSTATE_GENERAL_ERROR);
// Validate the secret and keep the session that originated the query alive.
ScopedSessionState session_handle(this);
const TUniqueId session_id = request_state->session_id();
const TUniqueId session_id = query_handle->session_id();
shared_ptr<SessionState> session;
HS2_RETURN_IF_ERROR(return_val,
session_handle.WithSession(
@@ -883,7 +883,7 @@ void ImpalaServer::FetchResults(TFetchResultsResp& return_val,
SQLSTATE_GENERAL_ERROR);
int32_t num_results = 0;
Status status = FetchInternal(request_state.get(), session.get(), request.maxRows,
Status status = FetchInternal(query_id, session.get(), request.maxRows,
fetch_first, &return_val, &num_results);
VLOG_ROW << "FetchResults(): query_id=" << PrintId(query_id)
@@ -912,23 +912,21 @@ void ImpalaServer::GetLog(TGetLogResp& return_val, const TGetLogReq& request) {
request.operationHandle.operationId, &query_id, &op_secret),
SQLSTATE_GENERAL_ERROR);
shared_ptr<ClientRequestState> request_state = GetClientRequestState(query_id);
if (UNLIKELY(request_state.get() == nullptr)) {
Status err = Status::Expected(TErrorCode::INVALID_QUERY_HANDLE, PrintId(query_id));
string err_msg = err.GetDetail();
HS2_RETURN_ERROR(return_val, err_msg, SQLSTATE_GENERAL_ERROR);
}
QueryHandle query_handle;
HS2_RETURN_IF_ERROR(
return_val, GetActiveQueryHandle(query_id, &query_handle), SQLSTATE_GENERAL_ERROR);
// GetLog doesn't have an associated session handle, so we presume that this request
// should keep alive the same session that orignated the query.
ScopedSessionState session_handle(this);
const TUniqueId session_id = request_state->session_id();
const TUniqueId session_id = query_handle->session_id();
HS2_RETURN_IF_ERROR(return_val,
session_handle.WithSession(session_id, SecretArg::Operation(op_secret, query_id)),
SQLSTATE_GENERAL_ERROR);
stringstream ss;
Coordinator* coord = request_state->GetCoordinator();
Coordinator* coord = query_handle->GetCoordinator();
if (coord != nullptr) {
// Report progress
ss << coord->progress().ToString() << "\n";
@@ -937,24 +935,24 @@ void ImpalaServer::GetLog(TGetLogResp& return_val, const TGetLogReq& request) {
{
// Take the lock to ensure that if the client sees a query_state == EXCEPTION, it is
// guaranteed to see the error query_status.
lock_guard<mutex> l(*request_state->lock());
Status query_status = request_state->query_status();
DCHECK_EQ(request_state->exec_state() == ClientRequestState::ExecState::ERROR,
lock_guard<mutex> l(*query_handle->lock());
Status query_status = query_handle->query_status();
DCHECK_EQ(query_handle->exec_state() == ClientRequestState::ExecState::ERROR,
!query_status.ok());
// If the query status is !ok, include the status error message at the top of the log.
if (!query_status.ok()) ss << query_status.GetDetail();
}
// Report analysis errors
ss << join(request_state->GetAnalysisWarnings(), "\n");
ss << join(query_handle->GetAnalysisWarnings(), "\n");
// Report queuing reason if the admission controller queued the query.
const string* admission_result = request_state->summary_profile()->GetInfoString(
const string* admission_result = query_handle->summary_profile()->GetInfoString(
AdmissionController::PROFILE_INFO_KEY_ADMISSION_RESULT);
if (admission_result != nullptr) {
if (*admission_result == AdmissionController::PROFILE_INFO_VAL_QUEUED) {
ss << AdmissionController::PROFILE_INFO_KEY_ADMISSION_RESULT << " : "
<< *admission_result << "\n";
const string* queued_reason = request_state->summary_profile()->GetInfoString(
const string* queued_reason = query_handle->summary_profile()->GetInfoString(
AdmissionController::PROFILE_INFO_KEY_LAST_QUEUED_REASON);
if (queued_reason != nullptr) {
ss << AdmissionController::PROFILE_INFO_KEY_LAST_QUEUED_REASON << " : "
@@ -1023,9 +1021,20 @@ void ImpalaServer::GetRuntimeProfile(
request.operationHandle.operationId, &query_id, &op_secret),
SQLSTATE_GENERAL_ERROR);
VLOG_RPC << "GetRuntimeProfile(): query_id=" << PrintId(query_id);
stringstream ss;
TRuntimeProfileTree thrift_profile;
rapidjson::Document json_profile(rapidjson::kObjectType);
// If the query was retried, fetch the profile for the most recent attempt of the query
// The original query profile should still be accessible via the web ui.
QueryHandle query_handle;
Status status = GetActiveQueryHandle(query_id, &query_handle);
if (LIKELY(status.ok())) {
query_id = query_handle->query_id();
}
HS2_RETURN_IF_ERROR(return_val,
GetRuntimeProfileOutput(query_id, GetEffectiveUser(*session), request.format, &ss,
&thrift_profile, &json_profile),

View File

@@ -32,6 +32,7 @@
#include "runtime/coordinator.h"
#include "runtime/exec-env.h"
#include "runtime/mem-tracker.h"
#include "runtime/query-driver.h"
#include "runtime/query-state.h"
#include "runtime/timestamp-value.h"
#include "runtime/timestamp-value.inline.h"
@@ -326,9 +327,9 @@ void ImpalaHttpHandler::QueryProfileJsonHandler(const Webserver::WebRequest& req
void ImpalaHttpHandler::InflightQueryIdsHandler(const Webserver::WebRequest& req,
Document* document) {
stringstream ss;
server_->client_request_state_map_.DoFuncForAllEntries(
[&](const std::shared_ptr<ClientRequestState>& request_state) {
ss << PrintId(request_state->query_id()) << "\n";
server_->query_driver_map_.DoFuncForAllEntries(
[&](const std::shared_ptr<QueryDriver>& query_driver) {
ss << PrintId(query_driver->GetActiveClientRequestState()->query_id()) << "\n";
});
document->AddMember(rapidjson::StringRef(Webserver::ENABLE_RAW_HTML_KEY), true,
document->GetAllocator());
@@ -416,8 +417,7 @@ void ImpalaHttpHandler::QueryStateToJson(const ImpalaServer::QueryStateRecord& r
Value progress_json(progress.c_str(), document->GetAllocator());
value->AddMember("progress", progress_json, document->GetAllocator());
Value state(_QueryState_VALUES_TO_NAMES.find(record.query_state)->second,
document->GetAllocator());
Value state(record.query_state.c_str(), document->GetAllocator());
value->AddMember("state", state, document->GetAllocator());
value->AddMember("rows_fetched", record.num_rows_fetched, document->GetAllocator());
@@ -432,7 +432,7 @@ void ImpalaHttpHandler::QueryStateToJson(const ImpalaServer::QueryStateRecord& r
}
// Waiting to be closed.
bool waiting = record.query_state == beeswax::QueryState::EXCEPTION ||
bool waiting = record.beeswax_query_state == beeswax::QueryState::EXCEPTION ||
record.all_rows_returned;
value->AddMember("waiting", waiting, document->GetAllocator());
value->AddMember("executing", !waiting, document->GetAllocator());
@@ -454,9 +454,10 @@ void ImpalaHttpHandler::QueryStateHandler(const Webserver::WebRequest& req,
set<ImpalaServer::QueryStateRecord, ImpalaServer::QueryStateRecordLessThan>
sorted_query_records;
server_->client_request_state_map_.DoFuncForAllEntries(
[&](const std::shared_ptr<ClientRequestState>& request_state) {
sorted_query_records.insert(ImpalaServer::QueryStateRecord(*request_state));
server_->query_driver_map_.DoFuncForAllEntries(
[&](const std::shared_ptr<QueryDriver>& query_driver) {
sorted_query_records.insert(
ImpalaServer::QueryStateRecord(*query_driver->GetActiveClientRequestState()));
});
unordered_set<TUniqueId> in_flight_query_ids;
@@ -805,12 +806,14 @@ void ImpalaHttpHandler::QueryBackendsHandler(
return;
}
shared_ptr<ClientRequestState> request_state = server_->GetClientRequestState(query_id);
if (request_state.get() == nullptr || request_state->GetCoordinator() == nullptr) {
return;
QueryHandle query_handle;
status = server_->GetQueryHandle(query_id, &query_handle);
if (status.ok()) {
if (query_handle->GetCoordinator() == nullptr) {
return;
}
query_handle->GetCoordinator()->BackendsToJson(document);
}
request_state->GetCoordinator()->BackendsToJson(document);
}
void ImpalaHttpHandler::QueryFInstancesHandler(
@@ -826,12 +829,14 @@ void ImpalaHttpHandler::QueryFInstancesHandler(
return;
}
shared_ptr<ClientRequestState> request_state = server_->GetClientRequestState(query_id);
if (request_state.get() == nullptr || request_state->GetCoordinator() == nullptr) {
return;
QueryHandle query_handle;
status = server_->GetQueryHandle(query_id, &query_handle);
if (status.ok()) {
if (query_handle->GetCoordinator() == nullptr) {
return;
}
query_handle->GetCoordinator()->FInstanceStatsToJson(document);
}
request_state->GetCoordinator()->FInstanceStatsToJson(document);
}
void ImpalaHttpHandler::QuerySummaryHandler(bool include_json_plan, bool include_summary,
@@ -856,30 +861,30 @@ void ImpalaHttpHandler::QuerySummaryHandler(bool include_json_plan, bool include
// Search the in-flight queries first, followed by the archived ones.
{
shared_ptr<ClientRequestState> request_state =
server_->GetClientRequestState(query_id);
if (request_state != nullptr) {
QueryHandle query_handle;
status = server_->GetQueryHandle(query_id, &query_handle);
if (status.ok()) {
found = true;
// If the query plan isn't generated, avoid waiting for the request
// state lock to be acquired, since it could potentially be an expensive
// call, if the table Catalog metadata loading is in progress. Instead
// update the caller that the plan information is unavailable.
if (request_state->exec_state() == ClientRequestState::ExecState::INITIALIZED) {
if (query_handle->exec_state() == ClientRequestState::ExecState::INITIALIZED) {
document->AddMember(
"plan_metadata_unavailable", "true", document->GetAllocator());
return;
}
lock_guard<mutex> l(*request_state->lock());
query_status = request_state->query_status();
stmt = request_state->sql_stmt();
plan = request_state->exec_request().query_exec_request.query_plan;
lock_guard<mutex> l(*(*query_handle).lock());
query_status = query_handle->query_status();
stmt = query_handle->sql_stmt();
plan = query_handle->exec_request().query_exec_request.query_plan;
if ((include_json_plan || include_summary)
&& request_state->GetCoordinator() != nullptr) {
request_state->GetCoordinator()->GetTExecSummary(&summary);
&& query_handle->GetCoordinator() != nullptr) {
query_handle->GetCoordinator()->GetTExecSummary(&summary);
}
if (include_json_plan) {
for (const TPlanExecInfo& plan_exec_info:
request_state->exec_request().query_exec_request.plan_exec_info) {
query_handle->exec_request().query_exec_request.plan_exec_info) {
for (const TPlanFragment& fragment: plan_exec_info.fragments) {
fragments.push_back(fragment);
}
@@ -1055,9 +1060,10 @@ void ImpalaHttpHandler::AdmissionStateHandler(
unsigned long num_backends;
};
unordered_map<string, vector<QueryInfo>> running_queries;
server_->client_request_state_map_.DoFuncForAllEntries([&running_queries](
const std::shared_ptr<ClientRequestState>& request_state) {
server_->query_driver_map_.DoFuncForAllEntries([&running_queries](
const std::shared_ptr<QueryDriver>& query_driver) {
// Make sure only queries past admission control are added.
ClientRequestState* request_state = query_driver->GetActiveClientRequestState();
auto query_state = request_state->exec_state();
if (query_state != ClientRequestState::ExecState::INITIALIZED
&& query_state != ClientRequestState::ExecState::PENDING

View File

@@ -67,6 +67,7 @@
#include "runtime/timestamp-value.h"
#include "runtime/timestamp-value.inline.h"
#include "runtime/tmp-file-mgr.h"
#include "runtime/query-driver.h"
#include "scheduling/admission-controller.h"
#include "service/cancellation-work.h"
#include "service/client-request-state.h"
@@ -466,15 +467,13 @@ ImpalaServer::ImpalaServer(ExecEnv* exec_env)
cancellation_thread_pool_.reset(new ThreadPool<CancellationWork>(
"impala-server", "cancellation-worker",
FLAGS_cancellation_thread_pool_size, MAX_CANCELLATION_QUEUE_SIZE,
bind<void>(&ImpalaServer::CancelFromThreadPool, this, _1, _2)));
bind<void>(&ImpalaServer::CancelFromThreadPool, this, _2)));
ABORT_IF_ERROR(cancellation_thread_pool_->Init());
unreg_thread_pool_.reset(new ThreadPool<std::shared_ptr<ClientRequestState>>(
"impala-server", "unregistration-worker",
FLAGS_unregistration_thread_pool_size, FLAGS_unregistration_thread_pool_queue_depth,
[this] (uint32_t thread_id, shared_ptr<ClientRequestState> crs) {
FinishUnregisterQuery(move(crs));
}));
unreg_thread_pool_.reset(new ThreadPool<QueryHandle>("impala-server",
"unregistration-worker", FLAGS_unregistration_thread_pool_size,
FLAGS_unregistration_thread_pool_queue_depth,
bind<void>(&ImpalaServer::FinishUnregisterQuery, this, _2)));
ABORT_IF_ERROR(unreg_thread_pool_->Init());
// Initialize a session expiry thread which blocks indefinitely until the first session
@@ -643,28 +642,29 @@ Status ImpalaServer::GetRuntimeProfileOutput(const TUniqueId& query_id,
DCHECK(output != nullptr);
// Search for the query id in the active query map
{
shared_ptr<ClientRequestState> request_state =
GetClientRequestState(query_id, /*return_unregistered=*/ true);
if (request_state.get() != nullptr) {
QueryHandle query_handle;
Status status = GetQueryHandle(query_id, &query_handle,
/*return_unregistered=*/ true);
if (status.ok()) {
// For queries in INITIALIZED state, the profile information isn't populated yet.
if (request_state->exec_state() == ClientRequestState::ExecState::INITIALIZED) {
if (query_handle->exec_state() == ClientRequestState::ExecState::INITIALIZED) {
return Status::Expected("Query plan is not ready.");
}
lock_guard<mutex> l(*request_state->lock());
RETURN_IF_ERROR(CheckProfileAccess(user, request_state->effective_user(),
request_state->user_has_profile_access()));
if (request_state->GetCoordinator() != nullptr) {
UpdateExecSummary(request_state);
lock_guard<mutex> l(*query_handle->lock());
RETURN_IF_ERROR(CheckProfileAccess(user, query_handle->effective_user(),
query_handle->user_has_profile_access()));
if (query_handle->GetCoordinator() != nullptr) {
UpdateExecSummary(query_handle);
}
if (format == TRuntimeProfileFormat::BASE64) {
RETURN_IF_ERROR(request_state->profile()->SerializeToArchiveString(output));
RETURN_IF_ERROR(query_handle->profile()->SerializeToArchiveString(output));
} else if (format == TRuntimeProfileFormat::THRIFT) {
request_state->profile()->ToThrift(thrift_output);
query_handle->profile()->ToThrift(thrift_output);
} else if (format == TRuntimeProfileFormat::JSON) {
request_state->profile()->ToJson(json_output);
query_handle->profile()->ToJson(json_output);
} else {
DCHECK_EQ(format, TRuntimeProfileFormat::STRING);
request_state->profile()->PrettyPrint(output);
query_handle->profile()->PrettyPrint(output);
}
return Status::OK();
}
@@ -710,19 +710,20 @@ Status ImpalaServer::GetExecSummary(const TUniqueId& query_id, const string& use
TExecSummary* result) {
// Search for the query id in the active query map.
{
shared_ptr<ClientRequestState> request_state =
GetClientRequestState(query_id, /*return_unregistered=*/ true);
if (request_state != nullptr) {
lock_guard<mutex> l(*request_state->lock());
RETURN_IF_ERROR(CheckProfileAccess(user, request_state->effective_user(),
request_state->user_has_profile_access()));
if (request_state->exec_state() == ClientRequestState::ExecState::PENDING) {
const string* admission_result = request_state->summary_profile()->GetInfoString(
QueryHandle query_handle;
Status status = GetQueryHandle(query_id, &query_handle,
/*return_unregistered=*/ true);
if (status.ok()) {
lock_guard<mutex> l(*query_handle->lock());
RETURN_IF_ERROR(CheckProfileAccess(user, query_handle->effective_user(),
query_handle->user_has_profile_access()));
if (query_handle->exec_state() == ClientRequestState::ExecState::PENDING) {
const string* admission_result = query_handle->summary_profile()->GetInfoString(
AdmissionController::PROFILE_INFO_KEY_ADMISSION_RESULT);
if (admission_result != nullptr) {
if (*admission_result == AdmissionController::PROFILE_INFO_VAL_QUEUED) {
result->__set_is_queued(true);
const string* queued_reason = request_state->summary_profile()->GetInfoString(
const string* queued_reason = query_handle->summary_profile()->GetInfoString(
AdmissionController::PROFILE_INFO_KEY_LAST_QUEUED_REASON);
if (queued_reason != nullptr) {
result->__set_queued_reason(*queued_reason);
@@ -730,13 +731,13 @@ Status ImpalaServer::GetExecSummary(const TUniqueId& query_id, const string& use
}
}
return Status::OK();
} else if (request_state->GetCoordinator() != nullptr) {
request_state->GetCoordinator()->GetTExecSummary(result);
} else if (query_handle->GetCoordinator() != nullptr) {
query_handle->GetCoordinator()->GetTExecSummary(result);
TExecProgress progress;
progress.__set_num_completed_scan_ranges(
request_state->GetCoordinator()->progress().num_complete());
query_handle->GetCoordinator()->progress().num_complete());
progress.__set_total_scan_ranges(
request_state->GetCoordinator()->progress().total());
query_handle->GetCoordinator()->progress().total());
// TODO: does this not need to be synchronized?
result->__set_progress(progress);
return Status::OK();
@@ -815,9 +816,9 @@ Status ImpalaServer::GetExecSummary(const TUniqueId& query_id, const string& use
}
}
void ImpalaServer::ArchiveQuery(ClientRequestState* query) {
void ImpalaServer::ArchiveQuery(const QueryHandle& query_handle) {
vector<uint8_t> compressed_profile;
Status status = query->profile()->Compress(&compressed_profile);
Status status = query_handle->profile()->Compress(&compressed_profile);
if (!status.ok()) {
// Didn't serialize the string. Continue with empty string.
LOG_EVERY_N(WARNING, 1000) << "Could not serialize profile to archive string "
@@ -829,7 +830,7 @@ void ImpalaServer::ArchiveQuery(ClientRequestState* query) {
// FLAGS_log_query_to_file will have been set to false
if (FLAGS_log_query_to_file) {
stringstream ss;
ss << UnixMillis() << " " << PrintId(query->query_id()) << " ";
ss << UnixMillis() << " " << PrintId(query_handle->query_id()) << " ";
Base64Encode(compressed_profile, &ss);
status = profile_logger_->AppendEntry(ss.str());
if (!status.ok()) {
@@ -847,15 +848,16 @@ void ImpalaServer::ArchiveQuery(ClientRequestState* query) {
// the query has already been closed (e.g. no more results can be fetched).
unique_ptr<QueryStateRecord> record = nullptr;
{
lock_guard<mutex> l(*query->fetch_rows_lock());
record = make_unique<QueryStateRecord>(*query, move(compressed_profile));
lock_guard<mutex> l(*query_handle->fetch_rows_lock());
record = make_unique<QueryStateRecord>(*query_handle, move(compressed_profile));
}
if (query_handle->GetCoordinator() != nullptr) {
query_handle->GetCoordinator()->GetTExecSummary(&record->exec_summary);
}
if (query->GetCoordinator() != nullptr)
query->GetCoordinator()->GetTExecSummary(&record->exec_summary);
{
lock_guard<mutex> l(query_log_lock_);
// Add record to the beginning of the log, and to the lookup index.
query_log_index_[query->query_id()] = record.get();
query_log_index_[query_handle->query_id()] = record.get();
query_log_.insert(query_log_.begin(), move(record));
if (FLAGS_query_log_size > -1 && FLAGS_query_log_size < query_log_.size()) {
@@ -915,9 +917,8 @@ void ImpalaServer::AddPoolConfiguration(TQueryCtx* ctx,
}
}
Status ImpalaServer::Execute(TQueryCtx* query_ctx,
shared_ptr<SessionState> session_state,
shared_ptr<ClientRequestState>* request_state) {
Status ImpalaServer::Execute(TQueryCtx* query_ctx, shared_ptr<SessionState> session_state,
QueryHandle* query_handle) {
PrepareQueryContext(query_ctx);
ScopedThreadContext debug_ctx(GetThreadDebugInfo(), query_ctx->query_id);
ImpaladMetrics::IMPALA_SERVER_NUM_QUERIES->Increment(1L);
@@ -927,38 +928,39 @@ Status ImpalaServer::Execute(TQueryCtx* query_ctx,
Redact(&stmt);
query_ctx->client_request.__set_redacted_stmt((const string) stmt);
bool registered_request_state;
Status status = ExecuteInternal(*query_ctx, session_state, &registered_request_state,
request_state);
if (!status.ok() && registered_request_state) {
discard_result(UnregisterQuery((*request_state)->query_id(), false, &status));
bool registered_query = false;
Status status = ExecuteInternal(*query_ctx, session_state, &registered_query,
query_handle);
if (!status.ok() && registered_query) {
UnregisterQueryDiscardResult((*query_handle)->query_id(), false, &status);
}
return status;
}
Status ImpalaServer::ExecuteInternal(
const TQueryCtx& query_ctx,
shared_ptr<SessionState> session_state,
bool* registered_request_state,
shared_ptr<ClientRequestState>* request_state) {
Status ImpalaServer::ExecuteInternal(const TQueryCtx& query_ctx,
shared_ptr<SessionState> session_state, bool* registered_query,
QueryHandle* query_handle) {
DCHECK(session_state != nullptr);
*registered_request_state = false;
DCHECK(query_handle != nullptr);
DCHECK(registered_query != nullptr);
*registered_query = false;
request_state->reset(new ClientRequestState(
query_ctx, exec_env_, exec_env_->frontend(), this, session_state));
// Create the QueryDriver for this query. CreateNewDriver creates the associated
// ClientRequestState as well.
QueryDriver::CreateNewDriver(this, query_handle, query_ctx, session_state);
(*request_state)->query_events()->MarkEvent("Query submitted");
(*query_handle)->query_events()->MarkEvent("Query submitted");
{
// Keep a lock on request_state so that registration and setting
// Keep a lock on query_handle so that registration and setting
// result_metadata are atomic.
lock_guard<mutex> l(*(*request_state)->lock());
lock_guard<mutex> l(*(*query_handle)->lock());
// register exec state as early as possible so that queries that
// take a long time to plan show up, and to handle incoming status
// reports before execution starts.
RETURN_IF_ERROR(RegisterQuery(session_state, *request_state));
*registered_request_state = true;
RETURN_IF_ERROR(RegisterQuery(query_ctx.query_id, session_state, query_handle));
*registered_query = true;
#ifndef NDEBUG
// Inject a sleep to simulate metadata loading pauses for tables. This
@@ -978,23 +980,23 @@ Status ImpalaServer::ExecuteInternal(
// Takes the TQueryCtx and calls into the frontend to initialize the TExecRequest for
// this query.
RETURN_IF_ERROR((*request_state)->InitExecRequest(query_ctx));
RETURN_IF_ERROR(query_handle->query_driver()->RunFrontendPlanner(query_ctx));
const TExecRequest& result = (*request_state)->exec_request();
(*request_state)->query_events()->MarkEvent("Planning finished");
(*request_state)->set_user_profile_access(result.user_has_profile_access);
(*request_state)->summary_profile()->AddEventSequence(
const TExecRequest& result = (*query_handle)->exec_request();
(*query_handle)->query_events()->MarkEvent("Planning finished");
(*query_handle)->set_user_profile_access(result.user_has_profile_access);
(*query_handle)->summary_profile()->AddEventSequence(
result.timeline.name, result.timeline);
(*request_state)->SetFrontendProfile(result.profile);
(*query_handle)->SetFrontendProfile(result.profile);
if (result.__isset.result_set_metadata) {
(*request_state)->set_result_metadata(result.result_set_metadata);
(*query_handle)->set_result_metadata(result.result_set_metadata);
}
}
VLOG(2) << "Execution request: "
<< ThriftDebugString((*request_state)->exec_request());
<< ThriftDebugString((*query_handle)->exec_request());
// start execution of query; also starts fragment status reports
RETURN_IF_ERROR((*request_state)->Exec());
RETURN_IF_ERROR((*query_handle)->Exec());
Status status = UpdateCatalogMetrics();
if (!status.ok()) {
VLOG_QUERY << "Couldn't update catalog metrics: " << status.GetDetail();
@@ -1057,32 +1059,31 @@ void ImpalaServer::PrepareQueryContext(const TNetworkAddress& backend_addr,
}
}
Status ImpalaServer::RegisterQuery(shared_ptr<SessionState> session_state,
const shared_ptr<ClientRequestState>& request_state) {
Status ImpalaServer::RegisterQuery(const TUniqueId& query_id,
shared_ptr<SessionState> session_state, QueryHandle* query_handle) {
lock_guard<mutex> l2(session_state->lock);
// The session wasn't expired at the time it was checked out and it isn't allowed to
// expire while checked out, so it must not be expired.
DCHECK(session_state->ref_count > 0 && !session_state->expired);
DCHECK_GT(session_state->ref_count, 0);
DCHECK(!session_state->expired);
// The session may have been closed after it was checked out.
if (session_state->closed) {
VLOG(1) << "RegisterQuery(): session has been closed, ignoring query.";
return Status::Expected("Session has been closed, ignoring query.");
}
const TUniqueId& query_id = request_state->query_id();
DCHECK_EQ(this, ExecEnv::GetInstance()->impala_server());
RETURN_IF_ERROR(
client_request_state_map_.AddClientRequestState(query_id, request_state));
query_driver_map_.AddQueryDriver(query_id, query_handle->query_driver()));
// Metric is decremented in UnregisterQuery().
ImpaladMetrics::NUM_QUERIES_REGISTERED->Increment(1L);
VLOG_QUERY << "Registered query query_id=" << PrintId(query_id)
<< " session_id=" << PrintId(request_state->session_id());
VLOG_QUERY << "Registered query query_id=" << PrintId(query_id) << " session_id="
<< PrintId(session_state->session_id);
return Status::OK();
}
Status ImpalaServer::SetQueryInflight(shared_ptr<SessionState> session_state,
const shared_ptr<ClientRequestState>& request_state) {
const TUniqueId& query_id = request_state->query_id();
Status ImpalaServer::SetQueryInflight(
shared_ptr<SessionState> session_state, const QueryHandle& query_handle) {
const TUniqueId& query_id = query_handle->query_id();
lock_guard<mutex> l(session_state->lock);
// The session wasn't expired at the time it was checked out and it isn't allowed to
// expire while checked out, so it must not be expired.
@@ -1098,16 +1099,16 @@ Status ImpalaServer::SetQueryInflight(shared_ptr<SessionState> session_state,
++session_state->total_queries;
// If the query has a timeout or time limit, schedule checks.
int32_t idle_timeout_s = request_state->query_options().query_timeout_s;
int32_t idle_timeout_s = query_handle->query_options().query_timeout_s;
if (FLAGS_idle_query_timeout > 0 && idle_timeout_s > 0) {
idle_timeout_s = min(FLAGS_idle_query_timeout, idle_timeout_s);
} else {
// Use a non-zero timeout, if one exists
idle_timeout_s = max(FLAGS_idle_query_timeout, idle_timeout_s);
}
int32_t exec_time_limit_s = request_state->query_options().exec_time_limit_s;
int64_t cpu_limit_s = request_state->query_options().cpu_limit_s;
int64_t scan_bytes_limit = request_state->query_options().scan_bytes_limit;
int32_t exec_time_limit_s = query_handle->query_options().exec_time_limit_s;
int64_t cpu_limit_s = query_handle->query_options().cpu_limit_s;
int64_t scan_bytes_limit = query_handle->query_options().scan_bytes_limit;
if (idle_timeout_s > 0 || exec_time_limit_s > 0 ||
cpu_limit_s > 0 || scan_bytes_limit > 0) {
lock_guard<mutex> l2(query_expiration_lock_);
@@ -1140,73 +1141,84 @@ Status ImpalaServer::SetQueryInflight(shared_ptr<SessionState> session_state,
return Status::OK();
}
void ImpalaServer::UpdateExecSummary(
std::shared_ptr<ClientRequestState> request_state) const {
DCHECK(request_state->GetCoordinator() != nullptr);
void ImpalaServer::UpdateExecSummary(const QueryHandle& query_handle) const {
DCHECK(query_handle->GetCoordinator() != nullptr);
TExecSummary t_exec_summary;
request_state->GetCoordinator()->GetTExecSummary(&t_exec_summary);
request_state->summary_profile()->SetTExecSummary(t_exec_summary);
query_handle->GetCoordinator()->GetTExecSummary(&t_exec_summary);
query_handle->summary_profile()->SetTExecSummary(t_exec_summary);
string exec_summary = PrintExecSummary(t_exec_summary);
request_state->summary_profile()->AddInfoStringRedacted("ExecSummary", exec_summary);
request_state->summary_profile()->AddInfoStringRedacted("Errors",
request_state->GetCoordinator()->GetErrorLog());
query_handle->summary_profile()->AddInfoStringRedacted("ExecSummary", exec_summary);
query_handle->summary_profile()->AddInfoStringRedacted("Errors",
query_handle->GetCoordinator()->GetErrorLog());
}
Status ImpalaServer::UnregisterQuery(const TUniqueId& query_id, bool check_inflight,
const Status* cause) {
VLOG_QUERY << "UnregisterQuery(): query_id=" << PrintId(query_id);
shared_ptr<ClientRequestState> request_state = GetClientRequestState(query_id);
if (request_state == nullptr) {
return Status::Expected(TErrorCode::INVALID_QUERY_HANDLE, PrintId(query_id));
}
// We want to do some of the query unregistration work synchronously. Finalize
// only succeeds for the first thread to call it to avoid multiple threads
// unregistering.
RETURN_IF_ERROR(request_state->Finalize(check_inflight, cause));
QueryHandle query_handle;
RETURN_IF_ERROR(GetActiveQueryHandle(query_id, &query_handle));
// Do the work of unregistration that needs to be done synchronously. Once
// Finalize() returns, the query is considered unregistered from the client's point of
// view. If Finalize() returns OK, this thread is responsible for doing the
// unregistration work. Finalize() succeeds for the first thread to call it to avoid
// multiple threads unregistering.
RETURN_IF_ERROR(
query_handle.query_driver()->Finalize(&query_handle, check_inflight, cause));
// Do the rest of the unregistration work in the background so that the client does
// not need to wait for profile serialization, etc.
unreg_thread_pool_->Offer(move(request_state));
unreg_thread_pool_->Offer(move(query_handle));
return Status::OK();
}
void ImpalaServer::FinishUnregisterQuery(shared_ptr<ClientRequestState>&& request_state) {
void ImpalaServer::FinishUnregisterQuery(const QueryHandle& query_handle) {
DCHECK_EQ(this, ExecEnv::GetInstance()->impala_server());
// Do all the finalization before removing the ClientRequestState from the map so that
// Do all the finalization before removing the QueryDriver from the map so that
// concurrent operations, e.g. GetRuntimeProfile() can find the query.
CloseClientRequestState(request_state);
// Make the ClientRequestState inaccessible. There is a time window where the query is
// both in 'client_request_state_map_' and 'query_locations_'.
Status status =
client_request_state_map_.DeleteClientRequestState(request_state->query_id());
DCHECK(status.ok()) << "CRS can only be deleted once: " << status.GetDetail();
CloseClientRequestState(query_handle);
// Make the QueryDriver inaccessible. There is a time window where the query is
// both in 'query_driver_map_' and 'query_locations_'.
Status status = query_handle.query_driver()->Unregister(&query_driver_map_);
string err_msg = "QueryDriver can only be deleted once: " + status.GetDetail();
DCHECK(status.ok()) << err_msg;
if (UNLIKELY(!status.ok())) LOG(ERROR) << status.GetDetail();
}
void ImpalaServer::CloseClientRequestState(
const std::shared_ptr<ClientRequestState>& request_state) {
int64_t duration_us = request_state->end_time_us() - request_state->start_time_us();
void ImpalaServer::UnregisterQueryDiscardResult(
const TUniqueId& query_id, bool check_inflight, const Status* cause) {
Status status = UnregisterQuery(query_id, check_inflight, cause);
if (!status.ok()) {
LOG(ERROR) << Substitute("Query de-registration for query_id={0} failed: {1}",
PrintId(query_id, cause->GetDetail()));
}
}
void ImpalaServer::CloseClientRequestState(const QueryHandle& query_handle) {
int64_t duration_us = query_handle->end_time_us() - query_handle->start_time_us();
int64_t duration_ms = duration_us / MICROS_PER_MILLI;
// duration_ms can be negative when the local timezone changes during query execution.
if (duration_ms >= 0) {
if (request_state->stmt_type() == TStmtType::DDL) {
if (query_handle->stmt_type() == TStmtType::DDL) {
ImpaladMetrics::DDL_DURATIONS->Update(duration_ms);
} else {
ImpaladMetrics::QUERY_DURATIONS->Update(duration_ms);
}
}
{
lock_guard<mutex> l(request_state->session()->lock);
request_state->session()->inflight_queries.erase(request_state->query_id());
lock_guard<mutex> l(query_handle->session()->lock);
query_handle->session()->inflight_queries.erase(query_handle->query_id());
}
if (request_state->GetCoordinator() != nullptr) {
UpdateExecSummary(request_state);
if (query_handle->GetCoordinator() != nullptr) {
UpdateExecSummary(query_handle);
}
if (request_state->schedule() != nullptr) {
if (query_handle->schedule() != nullptr) {
const PerBackendExecParams& per_backend_params =
request_state->schedule()->per_backend_exec_params();
query_handle->schedule()->per_backend_exec_params();
if (!per_backend_params.empty()) {
lock_guard<mutex> l(query_locations_lock_);
for (const auto& entry : per_backend_params) {
@@ -1216,12 +1228,12 @@ void ImpalaServer::CloseClientRequestState(
// thing. They will harmlessly race to remove the query from this map.
auto it = query_locations_.find(entry.second.be_desc.backend_id());
if (it != query_locations_.end()) {
it->second.query_ids.erase(request_state->query_id());
it->second.query_ids.erase(query_handle->query_id());
}
}
}
}
ArchiveQuery(request_state.get());
ArchiveQuery(query_handle);
ImpaladMetrics::NUM_QUERIES_REGISTERED->Increment(-1L);
}
@@ -1262,13 +1274,57 @@ Status ImpalaServer::UpdateCatalogMetrics() {
}
shared_ptr<QueryDriver> ImpalaServer::GetQueryDriver(
const TUniqueId& query_id, bool return_unregistered) {
DCHECK_EQ(this, ExecEnv::GetInstance()->impala_server());
ScopedShardedMapRef<std::shared_ptr<QueryDriver>> map_ref(query_id, &query_driver_map_);
DCHECK(map_ref.get() != nullptr);
auto entry = map_ref->find(query_id);
if (entry == map_ref->end()) return shared_ptr<QueryDriver>();
// This started_unregister() check can race with unregistration. It cannot prevent
// unregistration starting immediately after the value is loaded. This check, however,
// is sufficient to ensure that after a client operation has unregistered the request,
// subsequent operations won't spuriously find the request.
if (!return_unregistered && entry->second->finalized()) {
return shared_ptr<QueryDriver>();
}
return entry->second;
}
Status ImpalaServer::GetActiveQueryHandle(
const TUniqueId& query_id, QueryHandle* query_handle) {
DCHECK(query_handle != nullptr);
shared_ptr<QueryDriver> query_driver = GetQueryDriver(query_id);
if (UNLIKELY(query_driver == nullptr)) {
Status err = Status::Expected(TErrorCode::INVALID_QUERY_HANDLE, PrintId(query_id));
VLOG(1) << err.GetDetail();
return err;
}
query_handle->SetHandle(query_driver, query_driver->GetActiveClientRequestState());
return Status::OK();
}
Status ImpalaServer::GetQueryHandle(
const TUniqueId& query_id, QueryHandle* query_handle, bool return_unregistered) {
DCHECK(query_handle != nullptr);
shared_ptr<QueryDriver> query_driver = GetQueryDriver(query_id, return_unregistered);
if (UNLIKELY(query_driver == nullptr)) {
Status err = Status::Expected(TErrorCode::INVALID_QUERY_HANDLE, PrintId(query_id));
VLOG(1) << err.GetDetail();
return err;
}
query_handle->SetHandle(query_driver, query_driver->GetClientRequestState(query_id));
return Status::OK();
}
Status ImpalaServer::CancelInternal(const TUniqueId& query_id) {
VLOG_QUERY << "Cancel(): query_id=" << PrintId(query_id);
shared_ptr<ClientRequestState> request_state = GetClientRequestState(query_id);
if (request_state == nullptr) {
return Status::Expected(TErrorCode::INVALID_QUERY_HANDLE, PrintId(query_id));
}
RETURN_IF_ERROR(request_state->Cancel(/*check_inflight=*/ true, /*cause=*/ nullptr));
QueryHandle query_handle;
RETURN_IF_ERROR(GetActiveQueryHandle(query_id, &query_handle));
RETURN_IF_ERROR(
query_handle->Cancel(/*check_inflight=*/ true, /*cause=*/ nullptr));
return Status::OK();
}
@@ -1318,7 +1374,7 @@ Status ImpalaServer::CloseSessionInternal(const TUniqueId& session_id,
Status status = Status::Expected("Session closed");
for (const TUniqueId& query_id: inflight_queries) {
// TODO: deal with an error status
discard_result(UnregisterQuery(query_id, false, &status));
UnregisterQueryDiscardResult(query_id, false, &status);
}
// Reconfigure the poll period of session_maintenance_thread_ if necessary.
UnregisterSessionTimeout(session_state->session_timeout);
@@ -1438,18 +1494,55 @@ TQueryOptions ImpalaServer::SessionState::QueryOptions() {
return ret;
}
void ImpalaServer::CancelFromThreadPool(uint32_t thread_id,
const CancellationWork& cancellation_work) {
Status ImpalaServer::WaitForResults(const TUniqueId& query_id,
QueryHandle* query_handle, int64_t* block_on_wait_time_us,
bool* timed_out) {
// Make sure ClientRequestState::Wait() has completed before fetching rows. Wait()
// ensures that rows are ready to be fetched (e.g., Wait() opens
// ClientRequestState::output_exprs_, which are evaluated in
// ClientRequestState::FetchRows() below).
RETURN_IF_ERROR(GetActiveQueryHandle(query_id, query_handle));
BlockOnWait(*query_handle, timed_out, block_on_wait_time_us);
// After BlockOnWait returns, it is possible that the query did not time out, and that
// it was instead retried. In that case, wait until the query has been successfully
// retried and then call GetActiveQueryHandle, which should now return the
// ClientRequestState for the new query.
ClientRequestState::RetryState retry_state;
retry_state = (*query_handle)->retry_state();
if (retry_state == ClientRequestState::RetryState::RETRYING
|| retry_state == ClientRequestState::RetryState::RETRIED) {
(*query_handle)->WaitUntilRetried();
RETURN_IF_ERROR(GetActiveQueryHandle(query_id, query_handle));
// Call BlockOnWait and then DCHECK that the state is not RETRYING or RETRIED
BlockOnWait(*query_handle, timed_out, block_on_wait_time_us);
retry_state = (*query_handle)->retry_state();
DCHECK(retry_state != ClientRequestState::RetryState::RETRYING
&& retry_state != ClientRequestState::RetryState::RETRIED)
<< "Unexpected state: " << (*query_handle)->RetryStateToString(retry_state);
}
return Status::OK();
}
void ImpalaServer::BlockOnWait(QueryHandle& query_handle,
bool* timed_out, int64_t* block_on_wait_time_us) {
int64_t fetch_rows_timeout_us = query_handle->fetch_rows_timeout_us();
*timed_out =
!query_handle->BlockOnWait(fetch_rows_timeout_us, block_on_wait_time_us);
}
void ImpalaServer::CancelFromThreadPool(const CancellationWork& cancellation_work) {
const TUniqueId& query_id = cancellation_work.query_id();
shared_ptr<ClientRequestState> request_state = GetClientRequestState(query_id);
QueryHandle query_handle;
Status status = GetQueryHandle(query_id, &query_handle);
// Query was already unregistered.
if (request_state == nullptr) {
if (!status.ok()) {
VLOG_QUERY << "CancelFromThreadPool(): query " << PrintId(query_id)
<< " already unregistered.";
return;
}
DebugActionNoFail(request_state->query_options(), "QUERY_CANCELLATION_THREAD");
DebugActionNoFail(query_handle->query_options(), "QUERY_CANCELLATION_THREAD");
Status error;
switch (cancellation_work.cause()) {
case CancellationWorkCause::TERMINATED_BY_SERVER:
@@ -1459,7 +1552,7 @@ void ImpalaServer::CancelFromThreadPool(uint32_t thread_id,
// We only want to proceed with cancellation if the backends are still in use for
// the query.
vector<NetworkAddressPB> active_backends;
Coordinator* coord = request_state->GetCoordinator();
Coordinator* coord = query_handle->GetCoordinator();
if (coord == nullptr) {
// Query hasn't started yet - it still will run on all backends.
active_backends = cancellation_work.failed_backends();
@@ -1484,17 +1577,26 @@ void ImpalaServer::CancelFromThreadPool(uint32_t thread_id,
}
if (cancellation_work.unregister()) {
Status status = UnregisterQuery(cancellation_work.query_id(), true, &error);
if (!status.ok()) {
VLOG_QUERY << "Query de-registration (" << PrintId(cancellation_work.query_id())
<< ") failed";
}
UnregisterQueryDiscardResult(cancellation_work.query_id(), true, &error);
} else {
VLOG_QUERY << "CancelFromThreadPool(): cancelling query_id=" << PrintId(query_id);
Status status = request_state->Cancel(true, &error);
if (!status.ok()) {
VLOG_QUERY << "Query cancellation (" << PrintId(cancellation_work.query_id())
<< ") did not succeed: " << status.GetDetail();
// Retry queries that would otherwise be cancelled due to an impalad leaving the
// cluster. CancellationWorkCause::BACKEND_FAILED indicates that a backend running
// the query was removed from the cluster membership due to a statestore heartbeat
// timeout. Historically, this would cause the Coordinator to cancel all queries
// running on that backend. Now, Impala attempts to retry the queries instead of
// cancelling them.
bool was_retried = false;
if (cancellation_work.cause() == CancellationWorkCause::BACKEND_FAILED) {
query_handle.query_driver()->TryQueryRetry(&*query_handle, &error, &was_retried);
}
// If the query could not be retried, then cancel the query.
if (!was_retried) {
VLOG_QUERY << "CancelFromThreadPool(): cancelling query_id=" << PrintId(query_id);
Status status = query_handle->Cancel(true, &error);
if (!status.ok()) {
VLOG_QUERY << "Query cancellation (" << PrintId(cancellation_work.query_id())
<< ") did not succeed: " << status.GetDetail();
}
}
}
}
@@ -1862,56 +1964,62 @@ void ImpalaServer::BuildLocalBackendDescriptorInternal(BackendDescriptorPB* be_d
}
ImpalaServer::QueryStateRecord::QueryStateRecord(
const ClientRequestState& request_state, vector<uint8_t>&& compressed_profile)
const ClientRequestState& query_handle, vector<uint8_t>&& compressed_profile)
: compressed_profile(compressed_profile) {
Init(request_state);
Init(query_handle);
}
ImpalaServer::QueryStateRecord::QueryStateRecord(const ClientRequestState& request_state)
ImpalaServer::QueryStateRecord::QueryStateRecord(const ClientRequestState& query_handle)
: compressed_profile() {
Init(request_state);
Init(query_handle);
}
void ImpalaServer::QueryStateRecord::Init(const ClientRequestState& request_state) {
id = request_state.query_id();
const TExecRequest& request = request_state.exec_request();
void ImpalaServer::QueryStateRecord::Init(const ClientRequestState& query_handle) {
id = query_handle.query_id();
const TExecRequest& request = query_handle.exec_request();
const string* plan_str = request_state.summary_profile()->GetInfoString("Plan");
const string* plan_str = query_handle.summary_profile()->GetInfoString("Plan");
if (plan_str != nullptr) plan = *plan_str;
stmt = request_state.sql_stmt();
stmt = query_handle.sql_stmt();
stmt_type = request.stmt_type;
effective_user = request_state.effective_user();
default_db = request_state.default_db();
start_time_us = request_state.start_time_us();
end_time_us = request_state.end_time_us();
effective_user = query_handle.effective_user();
default_db = query_handle.default_db();
start_time_us = query_handle.start_time_us();
end_time_us = query_handle.end_time_us();
has_coord = false;
Coordinator* coord = request_state.GetCoordinator();
Coordinator* coord = query_handle.GetCoordinator();
if (coord != nullptr) {
num_complete_fragments = coord->progress().num_complete();
total_fragments = coord->progress().total();
has_coord = true;
}
query_state = request_state.BeeswaxQueryState();
num_rows_fetched = request_state.num_rows_fetched();
query_status = request_state.query_status();
beeswax_query_state = query_handle.BeeswaxQueryState();
ClientRequestState::RetryState retry_state = query_handle.retry_state();
if (retry_state == ClientRequestState::RetryState::NOT_RETRIED) {
query_state = _QueryState_VALUES_TO_NAMES.find(beeswax_query_state)->second;
} else {
query_state = query_handle.RetryStateToString(retry_state);
}
num_rows_fetched = query_handle.num_rows_fetched();
query_status = query_handle.query_status();
request_state.query_events()->ToThrift(&event_sequence);
query_handle.query_events()->ToThrift(&event_sequence);
// Save the query fragments so that the plan can be visualised.
for (const TPlanExecInfo& plan_exec_info:
request_state.exec_request().query_exec_request.plan_exec_info) {
query_handle.exec_request().query_exec_request.plan_exec_info) {
fragments.insert(fragments.end(),
plan_exec_info.fragments.begin(), plan_exec_info.fragments.end());
}
all_rows_returned = request_state.eos();
last_active_time_ms = request_state.last_active_ms();
all_rows_returned = query_handle.eos();
last_active_time_ms = query_handle.last_active_ms();
// For statement types other than QUERY/DML, show an empty string for resource pool
// to indicate that they are not subjected to admission control.
if (stmt_type == TStmtType::QUERY || stmt_type == TStmtType::DML) {
resource_pool = request_state.request_pool();
resource_pool = query_handle.request_pool();
}
user_has_profile_access = request_state.user_has_profile_access();
user_has_profile_access = query_handle.user_has_profile_access();
}
bool ImpalaServer::QueryStateRecordLessThan::operator() (
@@ -2195,23 +2303,28 @@ void ImpalaServer::UnregisterSessionTimeout(int32_t session_timeout) {
// 'queries_by_timestamp_' is stored in ascending order of deadline so we can
// break out of the loop and sleep as soon as we see a deadline in the future.
if (expiration_event->deadline > now) break;
shared_ptr<ClientRequestState> crs =
GetClientRequestState(expiration_event->query_id);
if (crs == nullptr || crs->is_expired()) {
// Query was deleted or expired already from a previous expiration event.
shared_ptr<QueryDriver> query_driver = GetQueryDriver(expiration_event->query_id);
if (query_driver == nullptr) {
// Query was deleted already from a previous expiration event
expiration_event = queries_by_timestamp_.erase(expiration_event);
continue;
}
ClientRequestState* crs = query_driver->GetActiveClientRequestState();
if (crs->is_expired()) {
// Query was expired already from a previous expiration event.
expiration_event = queries_by_timestamp_.erase(expiration_event);
continue;
}
// Check for CPU and scanned bytes limits
if (expiration_event->kind == ExpirationKind::RESOURCE_LIMIT) {
Status resource_status = CheckResourceLimits(crs.get());
Status resource_status = CheckResourceLimits(crs);
if (resource_status.ok()) {
queries_by_timestamp_.emplace(
ExpirationEvent{now + EXPIRATION_CHECK_INTERVAL_MS,
expiration_event->query_id, ExpirationKind::RESOURCE_LIMIT});
} else {
ExpireQuery(crs.get(), resource_status);
ExpireQuery(crs, resource_status);
}
expiration_event = queries_by_timestamp_.erase(expiration_event);
continue;
@@ -2222,7 +2335,7 @@ void ImpalaServer::UnregisterSessionTimeout(int32_t session_timeout) {
int32_t exec_time_limit_s = crs->query_options().exec_time_limit_s;
VLOG_QUERY << "Expiring query " << PrintId(expiration_event->query_id)
<< " due to execution time limit of " << exec_time_limit_s << "s.";
ExpireQuery(crs.get(),
ExpireQuery(crs,
Status::Expected(TErrorCode::EXEC_TIME_LIMIT_EXCEEDED,
PrintId(expiration_event->query_id),
PrettyPrinter::Print(exec_time_limit_s, TUnit::TIME_S)));
@@ -2263,7 +2376,7 @@ void ImpalaServer::UnregisterSessionTimeout(int32_t session_timeout) {
VLOG_QUERY << "Expiring query due to client inactivity: "
<< PrintId(expiration_event->query_id) << ", last activity was at: "
<< ToStringFromUnixMillis(crs->last_active_ms());
ExpireQuery(crs.get(),
ExpireQuery(crs,
Status::Expected(TErrorCode::INACTIVE_QUERY_EXPIRED,
PrintId(expiration_event->query_id),
PrettyPrinter::Print(idle_timeout_s, TUnit::TIME_S)));
@@ -2290,8 +2403,9 @@ void ImpalaServer::UnregisterSessionTimeout(int32_t session_timeout) {
<< "more than " << max_lag_ms << "ms.";
while (true) {
vector<CancellationWork> to_cancel;
client_request_state_map_.DoFuncForAllEntries(
[&](const std::shared_ptr<ClientRequestState>& request_state) {
query_driver_map_.DoFuncForAllEntries(
[&](const std::shared_ptr<QueryDriver>& query_driver) {
ClientRequestState* request_state = query_driver->GetActiveClientRequestState();
Coordinator* coord = request_state->GetCoordinator();
if (coord != nullptr) {
NetworkAddressPB address;
@@ -2309,7 +2423,7 @@ void ImpalaServer::UnregisterSessionTimeout(int32_t session_timeout) {
// We call Offer() outside of DoFuncForAllEntries() to ensure that if the
// cancellation_thread_pool_ queue is full, we're not blocked while holding one of the
// 'client_request_state_map_' shard locks.
// 'query_driver_map_' shard locks.
for (auto cancellation_work : to_cancel) {
cancellation_thread_pool_->Offer(cancellation_work);
}
@@ -2545,25 +2659,6 @@ void ImpalaServer::Join() {
}
}
shared_ptr<ClientRequestState> ImpalaServer::GetClientRequestState(
const TUniqueId& query_id, bool return_unregistered) {
DCHECK_EQ(this, ExecEnv::GetInstance()->impala_server());
ScopedShardedMapRef<std::shared_ptr<ClientRequestState>> map_ref(query_id,
&client_request_state_map_);
DCHECK(map_ref.get() != nullptr);
auto entry = map_ref->find(query_id);
if (entry == map_ref->end()) return shared_ptr<ClientRequestState>();
// This started_finalize() check can race with unregistration. It cannot prevent
// unregistration starting immediately after the value is loaded. This check, however,
// is sufficient to ensure that after a client operation has unregistered the request,
// subsequent operations won't spuriously find the request.
if (!return_unregistered && entry->second->started_finalize()) {
return shared_ptr<ClientRequestState>();
}
return entry->second;
}
Status ImpalaServer::CheckClientRequestSession(
SessionState* session, const std::string& client_request_effective_user,
const TUniqueId& query_id) {
@@ -2586,14 +2681,18 @@ void ImpalaServer::UpdateFilter(UpdateFilterResultPB* result,
const UpdateFilterParamsPB& params, RpcContext* context) {
DCHECK(params.has_query_id());
DCHECK(params.has_filter_id());
shared_ptr<ClientRequestState> client_request_state =
GetClientRequestState(ProtoToQueryId(params.query_id()));
if (client_request_state.get() == nullptr) {
LOG(INFO) << "Could not find client request state: "
QueryHandle query_handle;
Status status = GetQueryHandle(ProtoToQueryId(params.query_id()), &query_handle);
if (!status.ok()) {
LOG(INFO) << "Could not find query handle for query id: "
<< PrintId(ProtoToQueryId(params.query_id()));
return;
}
client_request_state->UpdateFilter(params, context);
ClientRequestState::RetryState retry_state = query_handle->retry_state();
if (retry_state != ClientRequestState::RetryState::RETRYING
&& retry_state != ClientRequestState::RetryState::RETRIED) {
query_handle->UpdateFilter(params, context);
}
}
Status ImpalaServer::CheckNotShuttingDown() const {

View File

@@ -38,7 +38,7 @@
#include "rpc/thrift-server.h"
#include "runtime/types.h"
#include "scheduling/query-schedule.h"
#include "service/client-request-state-map.h"
#include "service/query-driver-map.h"
#include "service/query-options.h"
#include "statestore/statestore-subscriber.h"
#include "util/condition-variable.h"
@@ -71,6 +71,8 @@ class TQueryOptions;
class TGetExecSummaryResp;
class TGetExecSummaryReq;
class ClientRequestState;
class QueryDriver;
struct QueryHandle;
class QuerySchedule;
class SimpleLogger;
class UpdateFilterParamsPB;
@@ -215,18 +217,18 @@ class ImpalaServer : public ImpalaServiceIf,
void Join();
/// ImpalaService rpcs: Beeswax API (implemented in impala-beeswax-server.cc)
virtual void query(beeswax::QueryHandle& query_handle, const beeswax::Query& query);
virtual void executeAndWait(beeswax::QueryHandle& query_handle,
virtual void query(beeswax::QueryHandle& beeswax_handle, const beeswax::Query& query);
virtual void executeAndWait(beeswax::QueryHandle& beeswax_handle,
const beeswax::Query& query, const beeswax::LogContextId& client_ctx);
virtual void explain(beeswax::QueryExplanation& query_explanation,
const beeswax::Query& query);
virtual void fetch(beeswax::Results& query_results,
const beeswax::QueryHandle& query_handle, const bool start_over,
const beeswax::QueryHandle& beeswax_handle, const bool start_over,
const int32_t fetch_size);
virtual void get_results_metadata(beeswax::ResultsMetadata& results_metadata,
const beeswax::QueryHandle& handle);
virtual void close(const beeswax::QueryHandle& handle);
virtual beeswax::QueryState::type get_state(const beeswax::QueryHandle& handle);
const beeswax::QueryHandle& beeswax_handle);
virtual void close(const beeswax::QueryHandle& beeswax_handle);
virtual beeswax::QueryState::type get_state(const beeswax::QueryHandle& beeswax_handle);
virtual void echo(std::string& echo_string, const std::string& input_string);
virtual void clean(const beeswax::LogContextId& log_context);
virtual void get_log(std::string& log, const beeswax::LogContextId& context);
@@ -244,18 +246,19 @@ class ImpalaServer : public ImpalaServiceIf,
/// ImpalaService rpcs: extensions over Beeswax (implemented in
/// impala-beeswax-server.cc)
virtual void Cancel(impala::TStatus& status, const beeswax::QueryHandle& query_id);
virtual void CloseInsert(impala::TDmlResult& dml_result,
const beeswax::QueryHandle& query_handle);
virtual void Cancel(
impala::TStatus& status, const beeswax::QueryHandle& beeswax_handle);
virtual void CloseInsert(
impala::TDmlResult& dml_result, const beeswax::QueryHandle& beeswax_handle);
/// Pings the Impala service and gets the server version string.
virtual void PingImpalaService(TPingImpalaServiceResp& return_val);
virtual void GetRuntimeProfile(std::string& profile_output,
const beeswax::QueryHandle& query_id);
const beeswax::QueryHandle& beeswax_handle);
virtual void GetExecSummary(impala::TExecSummary& result,
const beeswax::QueryHandle& query_id);
const beeswax::QueryHandle& beeswax_handle);
/// Performs a full catalog metadata reset, invalidating all table and database
/// metadata.
@@ -633,6 +636,7 @@ class ImpalaServer : public ImpalaServiceIf,
friend class ImpalaHttpHandler;
friend struct SessionState;
friend class ImpalaServerTest;
friend class QueryDriver;
boost::scoped_ptr<ImpalaHttpHandler> http_handler_;
@@ -642,13 +646,6 @@ class ImpalaServer : public ImpalaServiceIf,
static const char* SQLSTATE_GENERAL_ERROR;
static const char* SQLSTATE_OPTIONAL_FEATURE_NOT_IMPLEMENTED;
/// Return exec state for given query_id, or NULL if not found. If
/// 'return_unregistered' is true, queries that have started unregistration
/// may be returned. Otherwise queries that have started unregistration will
/// not be returned.
std::shared_ptr<ClientRequestState> GetClientRequestState(
const TUniqueId& query_id, bool return_unregistered=false);
/// Used in situations where the client provides a session ID and a query ID and the
/// caller needs to validate that the query can be accessed from the session. The two
/// arguments are the session obtained by looking up the session ID provided by the
@@ -667,24 +664,24 @@ class ImpalaServer : public ImpalaServiceIf,
/// Depending on the query type, this either submits the query to the admission
/// controller for performing async admission control or starts asynchronous execution
/// of query. Creates ClientRequestState (returned in exec_state), registers it and
/// calls ClientRequestState::Execute(). If it returns with an error status, exec_state
/// will be NULL and nothing will have been registered in client_request_state_map_.
/// session_state is a ptr to the session running this query and must have been checked
/// out.
/// of query. Creates ClientRequestState (returned in query_handle), registers it and
/// calls ClientRequestState::Execute(). If it returns with an error status,
/// query_driver->request_state will be NULL and nothing will have been registered in
/// query_driver_map_. session_state is a ptr to the session running this query and must
/// have been checked out.
Status Execute(TQueryCtx* query_ctx, std::shared_ptr<SessionState> session_state,
std::shared_ptr<ClientRequestState>* exec_state) WARN_UNUSED_RESULT;
QueryHandle* query_handle) WARN_UNUSED_RESULT;
/// Implements Execute() logic, but doesn't unregister query on error.
Status ExecuteInternal(const TQueryCtx& query_ctx,
std::shared_ptr<SessionState> session_state, bool* registered_exec_state,
std::shared_ptr<ClientRequestState>* exec_state) WARN_UNUSED_RESULT;
std::shared_ptr<SessionState> session_state, bool* registered_query,
QueryHandle* query_handle);
/// Registers the query exec state with client_request_state_map_ using the
/// globally unique query_id.
/// The caller must have checked out the session state.
Status RegisterQuery(std::shared_ptr<SessionState> session_state,
const std::shared_ptr<ClientRequestState>& exec_state) WARN_UNUSED_RESULT;
/// Registers the query with query_driver_map_ using the globally unique query_id. The
/// caller must have checked out the session state.
Status RegisterQuery(const TUniqueId& query_id,
std::shared_ptr<SessionState> session_state,
QueryHandle* query_handle) WARN_UNUSED_RESULT;
/// Adds the query to the set of in-flight queries for the session. The query remains
/// in-flight until the query is unregistered. Until a query is in-flight, an attempt
@@ -696,26 +693,31 @@ class ImpalaServer : public ImpalaServiceIf,
/// The query must have already been registered using RegisterQuery(). The caller
/// must have checked out the session state.
Status SetQueryInflight(std::shared_ptr<SessionState> session_state,
const std::shared_ptr<ClientRequestState>& exec_state) WARN_UNUSED_RESULT;
const QueryHandle& query_handle) WARN_UNUSED_RESULT;
/// Starts the process of unregistering the query. The query is cancelled on the
/// current thread, then asynchronously the query's entry is removed from
/// client_request_state_map_ and the session state's in-flight query list.
/// query_driver_map_ and the session state's in-flight query list.
/// If check_inflight is true, then return an error if the query
/// is not yet in-flight. Otherwise, proceed even if the query isn't yet in-flight (for
/// cleaning up after an error on the query issuing path).
Status UnregisterQuery(const TUniqueId& query_id, bool check_inflight,
const Status* cause = NULL) WARN_UNUSED_RESULT;
/// Delegates to UnregisterQuery. If UnregisterQuery returns an error Status, the
/// status is logged and then discarded.
void UnregisterQueryDiscardResult(
const TUniqueId& query_id, bool check_inflight, const Status* cause = NULL);
/// Unregisters the provided query, does all required finalization and removes it from
/// 'client_request_state_' map.
void FinishUnregisterQuery(std::shared_ptr<ClientRequestState>&& request_state);
/// 'query_driver_map_'.
void FinishUnregisterQuery(const QueryHandle& query_handle);
/// Performs finalization of 'request_state' before the request state is removed from
/// the server and deleted. Runs asynchronously after the request is reported done
/// to the client. Removes the query from the inflight queries list, updates
/// query_locations_, and archives the query.
void CloseClientRequestState(const std::shared_ptr<ClientRequestState>& request_state);
void CloseClientRequestState(const QueryHandle& query_handle);
/// Initiates query cancellation triggered by the user (i.e. deliberate cancellation).
/// Returns an error if query_id is not found or if the query is not yet in flight.
@@ -755,7 +757,7 @@ class ImpalaServer : public ImpalaServiceIf,
TExecSummary* result) WARN_UNUSED_RESULT;
/// Collect ExecSummary and update it to the profile in request_state
void UpdateExecSummary(std::shared_ptr<ClientRequestState> request_state) const;
void UpdateExecSummary(const QueryHandle& query_handle) const;
/// Initialize "default_configs_" to show the default values for ImpalaQueryOptions and
/// "support_start_over/false" to indicate that Impala does not support start over
@@ -804,7 +806,7 @@ class ImpalaServer : public ImpalaServiceIf,
/// Copies a query's state into the query log. Called immediately prior to a
/// ClientRequestState's deletion. Also writes the query profile to the profile log
/// on disk.
void ArchiveQuery(ClientRequestState* query);
void ArchiveQuery(const QueryHandle& query_handle);
/// Checks whether the given user is allowed to delegate as the specified do_as_user.
/// Returns OK if the authorization suceeds, otherwise returns an status with details
@@ -859,8 +861,17 @@ class ImpalaServer : public ImpalaServiceIf,
/// The number of rows fetched by the client
int64_t num_rows_fetched;
/// The state of the query as of this snapshot
beeswax::QueryState::type query_state;
/// The state of the query as of this snapshot. The possible values for the
/// query_state = union(beeswax::QueryState, ClientRequestState::RetryState). This is
/// necessary so that the query_state can accurately reflect if a query has been
/// retried or not. This string is not displayed in the runtime profiles, it is only
/// displayed on the /queries endpoint of the Web UI when listing out the state of
/// each query. This is necessary so that users can clearly see if a query has been
/// retried or not.
std::string query_state;
/// The beeswax::QueryState of the query as of this snapshot.
beeswax::QueryState::type beeswax_query_state;
/// Start and end time of the query, in Unix microseconds.
/// A query whose end_time_us is 0 indicates that it is an in-flight query.
@@ -911,19 +922,42 @@ class ImpalaServer : public ImpalaServiceIf,
bool operator() (const QueryStateRecord& lhs, const QueryStateRecord& rhs) const;
};
/// Returns the active QueryHandle for this query id. The QueryHandle contains the
/// active ClientRequestState. Returns an error Status if the query id cannot be found.
/// See QueryDriver for a description of active ClientRequestStates.
Status GetActiveQueryHandle(
const TUniqueId& query_id, QueryHandle* query_handle);
/// Similar to 'GetActiveQueryHandle' except it does not return the active handle, it
/// returns the handle directly associated with the given query id. Returns an error
/// Status if the query id cannot be found. See QueryDriver for a description of active
/// ClientRequestStates. See 'GetQueryDriver' for a description of the
/// 'return_unregistered' parameter.
Status GetQueryHandle(const TUniqueId& query_id, QueryHandle* query_handle,
bool return_unregistered = false);
/// Returns the QueryDriver for the given query_id, or nullptr if not found. If
/// 'return_unregistered' is true, queries that have started unregistration
/// may be returned. Otherwise queries that have started unregistration will
/// not be returned.
std::shared_ptr<QueryDriver> GetQueryDriver(
const TUniqueId& query_id, bool return_unregistered = false);
/// Beeswax private methods
/// Helper functions to translate between Beeswax and Impala structs
Status QueryToTQueryContext(const beeswax::Query& query, TQueryCtx* query_ctx)
WARN_UNUSED_RESULT;
void TUniqueIdToQueryHandle(const TUniqueId& query_id, beeswax::QueryHandle* handle);
void QueryHandleToTUniqueId(const beeswax::QueryHandle& handle, TUniqueId* query_id);
Status QueryToTQueryContext(
const beeswax::Query& query, TQueryCtx* query_ctx) WARN_UNUSED_RESULT;
void TUniqueIdToBeeswaxHandle(
const TUniqueId& query_id, beeswax::QueryHandle* beeswax_handle);
void BeeswaxHandleToTUniqueId(
const beeswax::QueryHandle& beeswax_handle, TUniqueId* query_id);
/// Helper function to raise BeeswaxException
[[noreturn]] void RaiseBeeswaxException(const std::string& msg, const char* sql_state);
/// Executes the fetch logic. Doesn't clean up the exec state if an error occurs.
Status FetchInternal(ClientRequestState* request_state, bool start_over,
Status FetchInternal(TUniqueId query_id, bool start_over,
int32_t fetch_size, beeswax::Results* query_results) WARN_UNUSED_RESULT;
/// Populate dml_result and clean up exec state. If the query
@@ -936,9 +970,9 @@ class ImpalaServer : public ImpalaServiceIf,
/// HiveServer2 private methods (implemented in impala-hs2-server.cc)
/// Starts the synchronous execution of a HiverServer2 metadata operation.
/// If the execution succeeds, an ClientRequestState will be created and registered in
/// client_request_state_map_. Otherwise, nothing will be registered in
/// client_request_state_map_ and an error status will be returned. As part of this
/// If the execution succeeds, a QueryDriver and ClientRequestState will be created
/// and registered in query_driver_map_. Otherwise, nothing will be registered in
/// query_driver_map_ and an error status will be returned. As part of this
/// call, the TMetadataOpRequest struct will be populated with the requesting user's
/// session state.
/// Returns a TOperationHandle and TStatus.
@@ -950,13 +984,23 @@ class ImpalaServer : public ImpalaServiceIf,
/// Executes the fetch logic for HiveServer2 FetchResults and stores result size in
/// 'num_results'. If fetch_first is true, then the query's state should be reset to
/// fetch from the beginning of the result set. Doesn't clean up 'request_state' if an
/// fetch from the beginning of the result set. Doesn't clean up exec state if an
/// error occurs.
Status FetchInternal(ClientRequestState* request_state, SessionState* session,
int32_t fetch_size, bool fetch_first,
Status FetchInternal(TUniqueId query_id, SessionState* session, int32_t fetch_size,
bool fetch_first,
apache::hive::service::cli::thrift::TFetchResultsResp* fetch_results,
int32_t* num_results) WARN_UNUSED_RESULT;
/// Setup the results cache. The results cache saves the results for a query so that
/// clients can re-read rows they have already fetched. The cache is used to support
/// TFetchOrientation::FETCH_FIRST option in TFetchResultsReq. Results cacheing only
/// saves results for as long as the query is open; results are not shared between
/// queries. This feature is useful for clients such as Hue, which allow users to
/// dynamically scroll through the results, but also allow users to download a file
/// containing all the results for a query.
Status SetupResultsCacheing(const QueryHandle& query_handle,
std::shared_ptr<SessionState> session, int64_t cache_num_rows) WARN_UNUSED_RESULT;
/// Helper functions to translate between HiveServer2 and Impala structs
/// Returns !ok() if handle.guid.size() or handle.secret.size() != 16
@@ -970,12 +1014,26 @@ class ImpalaServer : public ImpalaServiceIf,
const apache::hive::service::cli::thrift::TExecuteStatementReq execute_request,
TQueryCtx* query_ctx) WARN_UNUSED_RESULT;
/// Blocks until results are available. Handles any query retries that might occur
/// while waiting for results to be produced. It uses 'BlockOnWait' to wait for
/// results to be available. It is possible for a query to be retried while BlockOnWait
/// is running. If that happens, this method will first check if the query has been
/// retried. If it has been retried, it waits until the retry has started and then calls
/// BlockOnWait again, but with the QueryHandle of the retried query.
Status WaitForResults(const TUniqueId& query_id, QueryHandle* query_handle,
int64_t* block_on_wait_time_us, bool* timed_out);
/// Blocks until results from the given QueryHandle are ready to be fetched, or
/// until the given timeout is hit. 'timed_out' is set to true if the timeout
/// 'block_on_wait_time_us' was exceeded, false otherwise.
void BlockOnWait(
QueryHandle& query_handle, bool* timed_out, int64_t* block_on_wait_time_us);
/// Helper method to process cancellations that result from failed backends, called from
/// the cancellation thread pool. The cancellation_work contains the query id to cancel
/// and a cause listing the failed backends that led to cancellation. Calls
/// CancelInternal directly, but has a signature compatible with the thread pool.
void CancelFromThreadPool(uint32_t thread_id,
const CancellationWork& cancellation_work);
void CancelFromThreadPool(const CancellationWork& cancellation_work);
/// Helper method to add the pool name and query options to the query_ctx. Must be
/// called before ExecuteInternal() at which point the TQueryCtx is const and cannot
@@ -1041,7 +1099,7 @@ class ImpalaServer : public ImpalaServiceIf,
std::mutex query_log_lock_;
/// FIFO list of query records, which are written after the query finishes executing.
/// Queries may briefly have entries in 'query_log_' and 'client_request_state_map_'
/// Queries may briefly have entries in 'query_log_' and 'query_driver_map_'
/// while the query is being unregistered.
typedef std::list<std::unique_ptr<QueryStateRecord>> QueryLog;
QueryLog query_log_;
@@ -1081,8 +1139,8 @@ class ImpalaServer : public ImpalaServiceIf,
boost::scoped_ptr<ThreadPool<CancellationWork>> cancellation_thread_pool_;
/// Thread pool to unregister queries asynchronously from RPCs. FinishUnregisterQuery()
/// is called on all ClientRequestStates added to this pool.
boost::scoped_ptr<ThreadPool<std::shared_ptr<ClientRequestState>>> unreg_thread_pool_;
/// is called on all QueryHandles added to this pool.
boost::scoped_ptr<ThreadPool<QueryHandle>> unreg_thread_pool_;
/// Thread that runs SessionMaintenance. It will wake up periodically to check for
/// sessions which are idle for more their timeout values.
@@ -1101,11 +1159,11 @@ class ImpalaServer : public ImpalaServiceIf,
/// Thread that runs UnresponsiveBackendThread().
std::unique_ptr<Thread> unresponsive_backend_thread_;
/// A ClientRequestStateMap maps query ids to ClientRequestStates. The
/// ClientRequestStates are owned by the ImpalaServer and ClientRequestStateMap
/// references them using shared_ptr to allow asynchronous deletion.
/// The QueryDriverMap maps query ids to QueryDrivers. The QueryDrivers are owned by the
/// ImpalaServer and QueryDriverMap references them using shared_ptr to allow
/// asynchronous deletion.
///
/// ClientRequestStates are unregistered from the server as follows:
/// QueryDrivers are unregistered from the server as follows:
/// 1. UnregisterQuery() is called, which calls ClientRequestState::Finalize() to cancel
/// query execution and start the unregistration process. At this point the query is
/// considered unregistered from the client's point of view.
@@ -1113,8 +1171,8 @@ class ImpalaServer : public ImpalaServiceIf,
/// unregistration asynchronously.
/// 3. Additional cleanup work is done by CloseClientRequestState(), and an entry
/// is added to 'query_log_' for this query.
/// 4. The ClientRequestState is removed from this map.
ClientRequestStateMap client_request_state_map_;
/// 4. The QueryDriver is removed from this map.
QueryDriverMap query_driver_map_;
/// Default query options in the form of TQueryOptions and beeswax::ConfigVariable
TQueryOptions default_query_options_;
@@ -1286,6 +1344,12 @@ class ImpalaServer : public ImpalaServiceIf,
session->last_accessed_ms = UnixMillis();
}
/// Increment the session's reference counter.
inline void MarkSessionActive(std::shared_ptr<SessionState> session) {
std::lock_guard<std::mutex> l(session->lock);
++session->ref_count;
}
/// Associate the current connection context with the given session in
/// 'connection_to_sessions_map_' and 'SessionState::connections'.
void AddSessionToConnection(const TUniqueId& session_id, SessionState* session);

View File

@@ -15,7 +15,7 @@
// specific language governing permissions and limitations
// under the License.
#include "service/client-request-state-map.h"
#include "service/query-driver-map.h"
#include "gutil/strings/substitute.h"
#include "util/container-util.h"
@@ -26,9 +26,9 @@
namespace impala {
Status ClientRequestStateMap::AddClientRequestState(
const TUniqueId& query_id, shared_ptr<ClientRequestState> request_state) {
ScopedShardedMapRef<shared_ptr<ClientRequestState>> map_ref(query_id, this);
Status QueryDriverMap::AddQueryDriver(
const TUniqueId& query_id, std::shared_ptr<QueryDriver> query_driver) {
ScopedShardedMapRef<std::shared_ptr<QueryDriver>> map_ref(query_id, this);
DCHECK(map_ref.get() != nullptr);
auto entry = map_ref->find(query_id);
@@ -38,12 +38,12 @@ Status ClientRequestStateMap::AddClientRequestState(
return Status(ErrorMsg(TErrorCode::INTERNAL_ERROR,
strings::Substitute("query id $0 already exists", PrintId(query_id))));
}
map_ref->insert(make_pair(query_id, request_state));
map_ref->insert(make_pair(query_id, query_driver));
return Status::OK();
}
Status ClientRequestStateMap::DeleteClientRequestState(const TUniqueId& query_id) {
ScopedShardedMapRef<shared_ptr<ClientRequestState>> map_ref(query_id, this);
Status QueryDriverMap::DeleteQueryDriver(const TUniqueId& query_id) {
ScopedShardedMapRef<std::shared_ptr<QueryDriver>> map_ref(query_id, this);
DCHECK(map_ref.get() != nullptr);
auto entry = map_ref->find(query_id);
if (entry == map_ref->end()) {

View File

@@ -0,0 +1,44 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#pragma once
#include "common/status.h"
#include "util/sharded-query-map-util.h"
#include "util/unique-id-hash.h"
namespace impala {
class QueryDriver;
/// A ShardedQueryMap for QueryDrivers. Maps a query_id to its corresponding
/// QueryDriver. Provides helper methods to easily add and delete
/// QueryDrivers from a ShardedQueryMap. The QueryDrivers are non-const, so
/// users of this class can synchronize access to the QueryDrivers by creating a
/// ScopedShardedMapRef.
class QueryDriverMap : public ShardedQueryMap<std::shared_ptr<QueryDriver>> {
public:
/// Adds the given (query_id, query_driver) pair to the map. Returns an error Status
/// if the query id already exists in the map.
Status AddQueryDriver(
const TUniqueId& query_id, std::shared_ptr<QueryDriver> request_state);
/// Deletes the specified (query_id, query_driver) pair from the map. Returns an error
/// Status if the query_id cannot be found in the map.
Status DeleteQueryDriver(const TUniqueId& query_id);
};
}

View File

@@ -862,6 +862,9 @@ Status impala::SetQueryOption(const string& key, const string& value,
ParseMemValue(value, "broadcast bytes limit for join operations",
&broadcast_bytes_limit));
query_options->__set_broadcast_bytes_limit(broadcast_bytes_limit);
}
case TImpalaQueryOptions::RETRY_FAILED_QUERIES: {
query_options->__set_retry_failed_queries(IsTrue(value));
break;
}
case TImpalaQueryOptions::PREAGG_BYTES_LIMIT: {

View File

@@ -47,7 +47,7 @@ typedef std::unordered_map<string, beeswax::TQueryOptionLevel::type>
// time we add or remove a query option to/from the enum TImpalaQueryOptions.
#define QUERY_OPTS_TABLE\
DCHECK_EQ(_TImpalaQueryOptions_VALUES_TO_NAMES.size(),\
TImpalaQueryOptions::KUDU_SNAPSHOT_READ_TIMESTAMP_MICROS + 1);\
TImpalaQueryOptions::RETRY_FAILED_QUERIES + 1);\
REMOVED_QUERY_OPT_FN(abort_on_default_limit_exceeded, ABORT_ON_DEFAULT_LIMIT_EXCEEDED)\
QUERY_OPT_FN(abort_on_error, ABORT_ON_ERROR, TQueryOptionLevel::REGULAR)\
REMOVED_QUERY_OPT_FN(allow_unsupported_formats, ALLOW_UNSUPPORTED_FORMATS)\
@@ -198,7 +198,8 @@ typedef std::unordered_map<string, beeswax::TQueryOptionLevel::type>
QUERY_OPT_FN(preagg_bytes_limit, PREAGG_BYTES_LIMIT, TQueryOptionLevel::ADVANCED)\
QUERY_OPT_FN(max_cnf_exprs, MAX_CNF_EXPRS, TQueryOptionLevel::ADVANCED)\
QUERY_OPT_FN(kudu_snapshot_read_timestamp_micros, KUDU_SNAPSHOT_READ_TIMESTAMP_MICROS,\
TQueryOptionLevel::ADVANCED)
TQueryOptionLevel::ADVANCED)\
QUERY_OPT_FN(retry_failed_queries, RETRY_FAILED_QUERIES, TQueryOptionLevel::REGULAR)
;
/// Enforce practical limits on some query options to avoid undesired query state.

View File

@@ -58,7 +58,7 @@ Status ImpaladQueryExecutor::Setup() {
Status ImpaladQueryExecutor::Close() {
if (!query_in_progress_) return Status::OK();
try {
client_->iface()->close(query_handle_);
client_->iface()->close(beeswax_handle_);
} catch (BeeswaxException& e) {
stringstream ss;
ss << e.SQLState << ": " << e.message;
@@ -83,8 +83,8 @@ Status ImpaladQueryExecutor::Exec(
// does not provide a constant for it.
ResultsMetadata resultsMetadata;
try {
client_->iface()->executeAndWait(query_handle_, query, "");
client_->iface()->get_results_metadata(resultsMetadata, query_handle_);
client_->iface()->executeAndWait(beeswax_handle_, query, "");
client_->iface()->get_results_metadata(resultsMetadata, beeswax_handle_);
} catch (BeeswaxException& e) {
stringstream ss;
ss << e.SQLState << ": " << e.message;
@@ -105,7 +105,7 @@ Status ImpaladQueryExecutor::FetchResult(string* row) {
// from ImpalaServer
if (!query_results_.__isset.data || current_row_ >= query_results_.data.size()) {
try {
client_->iface()->fetch(query_results_, query_handle_, false, 0);
client_->iface()->fetch(query_results_, beeswax_handle_, false, 0);
} catch (BeeswaxException& e) {
stringstream ss;
ss << e.SQLState << ": " << e.message;

View File

@@ -101,7 +101,7 @@ class ImpaladQueryExecutor {
std::vector<std::string> exec_options_;
/// Beeswax query handle and result
beeswax::QueryHandle query_handle_;
beeswax::QueryHandle beeswax_handle_;
beeswax::Results query_results_;
beeswax::QueryExplanation query_explanation_;

View File

@@ -419,6 +419,9 @@ struct TQueryOptions {
// See comment in ImpalaService.thrift
102: optional i64 kudu_snapshot_read_timestamp_micros = 0;
// See comment in ImpalaService.thrift
103: optional bool retry_failed_queries = false;
}
// Impala currently has two types of sessions: Beeswax and HiveServer2

View File

@@ -520,6 +520,13 @@ enum TImpalaQueryOptions {
// Set the timestamp for Kudu snapshot reads in Unix time micros. Only valid if
// KUDU_READ_MODE is set to READ_AT_SNAPSHOT.
KUDU_SNAPSHOT_READ_TIMESTAMP_MICROS = 101
// Transparently retry queries that fail due to cluster membership changes. A cluster
// membership change includes blacklisting a node and the statestore detecting that a
// node has been removed from the cluster membership. From Impala's perspective, a
// retried query is a brand new query. From the client perspective, requests for the
// failed query are transparently re-routed to the new query.
RETRY_FAILED_QUERIES = 102
}
// The summary of a DML statement.

View File

@@ -478,14 +478,16 @@ class ImpaladProcess(BaseImpalaProcess):
def __get_hs2_http_port(self):
return int(self._get_port('hs2_http_port', DEFAULT_HS2_HTTP_PORT))
def start(self, wait_until_ready=True):
"""Starts the impalad and waits until the service is ready to accept connections."""
def start(self, wait_until_ready=True, timeout=30):
"""Starts the impalad and waits until the service is ready to accept connections.
'timeout' is the amount of time to wait for the Impala server to be in the
ready state."""
restart_args = self.cmd[1:]
LOG.info("Starting Impalad process with args: {0}".format(restart_args))
run_daemon("impalad", restart_args)
if wait_until_ready:
self.service.wait_for_metric_value('impala-server.ready',
expected_value=1, timeout=30)
expected_value=1, timeout=timeout)
def wait_for_catalog(self):
"""Waits for a catalog copy to be received by the impalad. When its received,

View File

@@ -224,8 +224,14 @@ class ImpaladService(BaseImpalaService):
return None
def get_in_flight_queries(self, timeout=30, interval=1):
"""Returns the number of in flight queries."""
return self.get_queries_json()['in_flight_queries']
def get_completed_queries(self, timeout=30, interval=1):
"""Returns the number of completed queries."""
result = json.loads(self.read_debug_webpage('queries?json', timeout, interval))
return result['completed_queries']
def _get_pool_counter(self, pool_name, counter_name, timeout=30, interval=1):
"""Returns the value of the field 'counter_name' in pool 'pool_name' or 0 if the pool
doesn't exist."""

View File

@@ -0,0 +1,563 @@
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
# TODO: Query retries can be triggered in several different ways, and some tests
# pick one approach for no particular reason, try to consolidate the different
# ways that retries are triggered.
# TODO: Re-factor tests into multiple classes.
# TODO: Add a test that cancels queries while a retry is running
import pytest
import re
import time
from random import randint
from RuntimeProfile.ttypes import TRuntimeProfileFormat
from tests.beeswax.impala_beeswax import ImpalaBeeswaxException
from tests.common.impala_test_suite import ImpalaTestSuite
from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
from tests.common.errors import Timeout
class TestQueryRetries(CustomClusterTestSuite):
# A query that shuffles a lot of data. Useful when testing query retries since it
# ensures that a query fails during a TransmitData RPC. The RPC failure will cause the
# target impalad to be blacklisted and the query to be retried. The query also has to
# run long enough so that it fails when an impalad is killed.
_shuffle_heavy_query = "select * from tpch.lineitem t1, tpch.lineitem t2 where " \
"t1.l_orderkey = t2.l_orderkey order by t1.l_orderkey, t2.l_orderkey limit 1"
_shuffle_heavy_query_results = "1\t155190\t7706\t1\t17.00\t21168.23\t0.04\t0.02\tN" \
"\tO\t1996-03-13\t1996-02-12\t1996-03-22\tDELIVER IN PERSON\tTRUCK" \
"\tegular courts above the\t1\t15635\t638\t6\t32.00\t49620.16\t0.07\t0.02\tN\tO" \
"\t1996-01-30\t1996-02-07\t1996-02-03\tDELIVER IN PERSON\tMAIL\tarefully slyly ex"
@classmethod
def get_workload(cls):
return 'functional-query'
@pytest.mark.execute_serially
def test_retries_from_cancellation_pool(self, cursor):
"""Tests that queries are retried instead of cancelled if one of the nodes leaves the
cluster. The retries are triggered by the cancellation pool in the ImpalaServer. The
cancellation pool listens for updates from the statestore and kills all queries that
are running on any nodes that are no longer part of the cluster membership."""
# The following query executes slowly, and does minimal TransmitData RPCs, so it is
# likely that the statestore detects that the impalad has been killed before a
# TransmitData RPC has occurred.
query = "select count(*) from functional.alltypes where bool_col = sleep(50)"
# Launch the query, wait for it to start running, and then kill an impalad.
handle = self.execute_query_async(query,
query_options={'retry_failed_queries': 'true'})
self.wait_for_state(handle, self.client.QUERY_STATES['RUNNING'], 60)
# Kill a random impalad (but not the one executing the actual query).
self.__kill_random_impalad()
# Validate the query results.
results = self.client.fetch(query, handle)
assert results.success
assert len(results.data) == 1
assert int(results.data[0]) == 3650
# Validate the state of the runtime profiles.
retried_runtime_profile = self.client.get_runtime_profile(handle)
self.__validate_runtime_profiles(retried_runtime_profile, handle.get_handle().id)
# Validate the state of the web ui. The query must be closed before validating the
# state since it asserts that no queries are in flight.
self.client.close_query(handle)
self.__validate_web_ui_state()
@pytest.mark.execute_serially
@CustomClusterTestSuite.with_args(
statestored_args="-statestore_heartbeat_frequency_ms=1000")
def test_kill_impalad_expect_retry(self):
"""Launch a query, wait for it to start running, kill a random impalad and then
validate that the query has successfully been retried. Increase the statestore
heartbeat frequency so that the query actually fails during execution. Otherwise, it
is possible the statestore detects that the killed impalad has crashed before the
query does. This is necessary since this test specifically attempts to test the code
that retries queries when a query fails mid-execution."""
# Launch a query, it should be retried.
handle = self.execute_query_async(self._shuffle_heavy_query,
query_options={'retry_failed_queries': 'true'})
self.wait_for_state(handle, self.client.QUERY_STATES['RUNNING'], 60)
# Kill a random impalad.
killed_impalad = self.__kill_random_impalad()
# Assert that the query succeeded and returned the correct results.
results = self.client.fetch(self._shuffle_heavy_query, handle)
assert results.success
assert len(results.data) == 1
assert self._shuffle_heavy_query_results in results.data[0]
# The runtime profile of the retried query.
retried_runtime_profile = self.client.get_runtime_profile(handle)
# Assert that the killed impalad shows up in the list of blacklisted executors from
# the runtime profile.
self.__assert_executors_blacklisted(killed_impalad, retried_runtime_profile)
# Validate the state of the runtime profiles.
self.__validate_runtime_profiles(retried_runtime_profile, handle.get_handle().id)
# Validate the state of the web ui. The query must be closed before validating the
# state since it asserts that no queries are in flight.
self.client.close_query(handle)
self.__validate_web_ui_state()
# Assert that the web ui shows all queries are complete.
completed_queries = self.cluster.get_first_impalad().service.get_completed_queries()
# Assert that the most recently completed query is the retried query and it is marked
# as 'FINISHED.
assert completed_queries[0]['state'] == 'FINISHED'
assert completed_queries[0]['query_id'] == self.__get_query_id_from_profile(
retried_runtime_profile)
# Assert that the second most recently completed query is the original query and it is
# marked as 'RETRIED'.
assert completed_queries[1]['state'] == 'RETRIED'
assert completed_queries[1]['query_id'] == handle.get_handle().id
@pytest.mark.execute_serially
@CustomClusterTestSuite.with_args(
statestored_args="-statestore_heartbeat_frequency_ms=1000")
def test_kill_impalad_expect_retries(self):
"""Similar to 'test_kill_impalad_expect_retry' except it runs multiple queries in
parallel and then kills an impalad. Several of the code comments in
'test_kill_impalad_expect_retry' apply here as well."""
# Launch a set of concurrent queries.
num_concurrent_queries = 3
handles = []
for _ in xrange(num_concurrent_queries):
handle = self.execute_query_async(self._shuffle_heavy_query,
query_options={'retry_failed_queries': 'true'})
handles.append(handle)
# Wait for each query to start running.
running_state = self.client.QUERY_STATES['RUNNING']
map(lambda handle: self.wait_for_state(handle, running_state, 60), handles)
# Kill a random impalad.
killed_impalad = self.__kill_random_impalad()
# Fetch and validate the results from each query.
for handle in handles:
results = self.client.fetch(self._shuffle_heavy_query, handle)
assert results.success
assert len(results.data) == 1
assert self._shuffle_heavy_query_results in results.data[0]
# Validate the runtime profiles of each query.
for handle in handles:
retried_runtime_profile = self.client.get_runtime_profile(handle)
self.__assert_executors_blacklisted(killed_impalad, retried_runtime_profile)
self.__validate_runtime_profiles(retried_runtime_profile, handle.get_handle().id)
self.client.close_query(handle)
# Validate the state of the Web UI.
self.__validate_web_ui_state()
@pytest.mark.execute_serially
@CustomClusterTestSuite.with_args(
statestored_args="-statestore_heartbeat_frequency_ms=60000")
def test_retry_exec_rpc_failure(self):
"""Test ExecFInstance RPC failures. Set a really high statestort heartbeat frequency
so that killed impalads are not removed from the cluster membership. This will cause
Impala to still attempt an Exec RPC to the failed node, which should trigger a
retry."""
impalad_service = self.cluster.get_first_impalad().service
# Kill an impalad, and run a query. The query should be retried.
killed_impalad = self.__kill_random_impalad()
query = "select count(*) from tpch_parquet.lineitem"
handle = self.execute_query_async(query,
query_options={'retry_failed_queries': 'true'})
self.wait_for_state(handle, self.client.QUERY_STATES['FINISHED'], 60)
# Validate that the query was retried.
self.__validate_runtime_profiles_from_service(impalad_service, handle)
# Assert that the query succeeded and returned the correct results.
results = self.client.fetch(query, handle)
assert results.success
assert len(results.data) == 1
assert "6001215" in results.data[0]
# The runtime profile of the retried query.
retried_runtime_profile = self.client.get_runtime_profile(handle)
# Assert that the killed impalad shows up in the list of blacklisted executors from
# the runtime profile.
self.__assert_executors_blacklisted(killed_impalad, retried_runtime_profile)
# Validate the state of the runtime profiles.
self.__validate_runtime_profiles(retried_runtime_profile, handle.get_handle().id)
# Validate the state of the web ui. The query must be closed before validating the
# state since it asserts that no queries are in flight.
self.client.close_query(handle)
self.__validate_web_ui_state()
@pytest.mark.execute_serially
@CustomClusterTestSuite.with_args(
statestored_args="-statestore_heartbeat_frequency_ms=1000")
def test_multiple_retries(self):
"""Test that a query can only be retried once, and that if the retry attempt fails,
it fails correctly and with the right error message. Multiple retry attempts are
triggered by killing multiple impalads. The final attempt at retrying the query
should indicate that the error was retryable, and that the max retry limit was
exceeded."""
# Launch a query, it should be retried.
handle = self.execute_query_async(self._shuffle_heavy_query,
query_options={'retry_failed_queries': 'true'})
self.wait_for_state(handle, self.client.QUERY_STATES['RUNNING'], 60)
# Kill one impalad so that a retry is triggered.
killed_impalad = self.cluster.impalads[1]
killed_impalad.kill()
# Wait until the retry is running.
self.__wait_until_retried(handle)
# Kill another impalad so that another retry is attempted.
self.cluster.impalads[2].kill()
# Wait until the query fails.
self.wait_for_state(handle, self.client.QUERY_STATES['EXCEPTION'], 60)
# The runtime profile of the retried query, need to retrieve the profile
# before fetching results, since the failed fetch attempt will close the
# query handle.
retried_runtime_profile = self.client.get_runtime_profile(handle)
# Assert that the query failed, since a query can only be retried once.
try:
self.client.fetch(self._shuffle_heavy_query, handle)
assert False
except ImpalaBeeswaxException, e:
assert "Max retry limit was hit. Query was retried 1 time(s)." in str(e)
# Assert that the killed impalad shows up in the list of blacklisted executors from
# the runtime profile.
self.__assert_executors_blacklisted(killed_impalad, retried_runtime_profile)
# Assert that the query id of the original query is in the runtime profile of the
# retried query.
self.__validate_original_id_in_profile(retried_runtime_profile,
handle.get_handle().id)
# Validate the state of the web ui. The query must be closed before validating the
# state since it asserts that no queries are in flight.
self.__validate_web_ui_state()
@pytest.mark.execute_serially
def test_retry_fetched_rows(self):
"""Test that query retries are not triggered if some rows have already been
fetched. Run a query, fetch some rows from it, kill one of the impalads that is
running the query, and the validate that another fetch request fails."""
query = "select * from functional.alltypes where bool_col = sleep(500)"
handle = self.execute_query_async(query,
query_options={'retry_failed_queries': 'true', 'batch_size': '1'})
self.wait_for_state(handle, self.client.QUERY_STATES['FINISHED'], 60)
self.client.fetch(query, handle, max_rows=1)
self.cluster.impalads[1].kill()
time.sleep(5)
# Assert than attempt to fetch from the query handle fails.
try:
self.client.fetch(query, handle)
assert False
except Exception, e:
assert "Failed due to unreachable impalad" in str(e)
@pytest.mark.execute_serially
@CustomClusterTestSuite.with_args(
statestored_args="-statestore_heartbeat_frequency_ms=60000")
def test_retry_query_cancel(self):
"""Trigger a query retry, and then cancel the retried query. Validate that the
cancelled query fails with the correct error message. Set a really high statestore
heartbeat frequency so that killed impalads are not removed from the cluster
membership."""
impalad_service = self.cluster.get_first_impalad().service
# Kill an impalad, and run a query. The query should be retried.
self.cluster.impalads[1].kill()
query = "select count(*) from tpch_parquet.lineitem"
handle = self.execute_query_async(query,
query_options={'retry_failed_queries': 'true'})
self.wait_for_state(handle, self.client.QUERY_STATES['FINISHED'], 60)
# Validate that the query was retried.
self.__validate_runtime_profiles_from_service(impalad_service, handle)
# Cancel the query.
self.client.cancel(handle)
# Assert than attempt to fetch from the query handle fails with a cancellation
# error
try:
self.client.fetch(query, handle)
assert False
except Exception, e:
assert "Cancelled" in str(e)
@pytest.mark.execute_serially
@CustomClusterTestSuite.with_args(
statestored_args="-statestore_heartbeat_frequency_ms=60000")
def test_retry_query_timeout(self):
"""Trigger a query retry, and then leave the query handle open until the
'query_timeout_s' causes the handle to be closed. Assert that the runtime profile of
the original and retried queries are correct, and that the 'num-queries-expired'
metric is properly incremented. Set a really high statestore heartbeat frequency so
that killed impalads are not removed from the cluster membership."""
impalad_service = self.cluster.get_first_impalad().service
# Kill an impalad, and run a query. The query should be retried.
self.cluster.impalads[1].kill()
query = "select count(*) from tpch_parquet.lineitem"
handle = self.execute_query_async(query,
query_options={'retry_failed_queries': 'true', 'query_timeout_s': '1'})
self.wait_for_state(handle, self.client.QUERY_STATES['EXCEPTION'], 60)
# Wait for the query timeout to expire the query handle.
time.sleep(5)
# Validate that the query was retried.
self.__validate_runtime_profiles_from_service(impalad_service, handle)
# Assert than attempt to fetch from the query handle fails with a query expired
# error.
try:
self.client.fetch(query, handle)
assert False
except Exception, e:
assert "expired due to client inactivity" in str(e)
# Assert that the impalad metrics show one expired query.
assert impalad_service.get_metric_value('impala-server.num-queries-expired') == 1
@pytest.mark.execute_serially
@CustomClusterTestSuite.with_args(impalad_args="--idle_session_timeout=1",
statestored_args="--statestore_heartbeat_frequency_ms=60000")
def test_retry_query_session_timeout(self):
"""Similar to 'test_retry_query_timeout' except with an idle session timeout."""
self.close_impala_clients()
impalad_service = self.cluster.get_first_impalad().service
# Kill an impalad, and run a query. The query should be retried.
self.cluster.impalads[1].kill()
query = "select count(*) from tpch_parquet.lineitem"
client = self.cluster.get_first_impalad().service.create_beeswax_client()
client.set_configuration({'retry_failed_queries': 'true'})
handle = client.execute_async(query)
self.wait_for_state(handle, client.QUERY_STATES['FINISHED'], 60, client=client)
# Wait for the idle session timeout to expire the session.
time.sleep(5)
# Validate that the query was retried.
self.__validate_runtime_profiles_from_service(impalad_service, handle)
# Assert than attempt to fetch from the query handle fails with a session expired
# error.
try:
client.fetch(query, handle)
except Exception, e:
assert "Client session expired" in str(e)
# Assert that the impalad metrics show one expired session.
assert impalad_service.get_metric_value('impala-server.num-sessions-expired') == 1
@pytest.mark.execute_serially
@CustomClusterTestSuite.with_args(
statestored_args="-statestore_heartbeat_frequency_ms=60000")
def test_retry_query_hs2(self):
"""Test query retries with the HS2 protocol. Enable the results set cache as well and
test that query retries work with the results cache."""
self.cluster.impalads[1].kill()
query = "select count(*) from tpch_parquet.lineitem"
self.hs2_client.set_configuration({'retry_failed_queries': 'true'})
self.hs2_client.set_configuration_option('impala.resultset.cache.size', '1024')
handle = self.hs2_client.execute_async(query)
self.wait_for_state(handle, 'FINISHED_STATE', 60, client=self.hs2_client)
results = self.hs2_client.fetch(query, handle)
assert results.success
assert len(results.data) == 1
assert int(results.data[0]) == 6001215
# Validate the state of the runtime profiles.
retried_runtime_profile = self.hs2_client.get_runtime_profile(handle,
TRuntimeProfileFormat.STRING)
self.__validate_runtime_profiles(retried_runtime_profile,
self.hs2_client.get_query_id(handle))
self.impalad_test_service.wait_for_metric_value(
'impala-server.resultset-cache.total-num-rows', 1, timeout=60)
self.hs2_client.close_query(handle)
def __validate_runtime_profiles_from_service(self, impalad_service, handle):
"""Wrapper around '__validate_runtime_profiles' that first fetches the retried profile
from the web ui."""
original_profile = impalad_service.read_query_profile_page(handle.get_handle().id)
retried_query_id = self.__get_retried_query_id_from_profile(original_profile)
retried_profile = impalad_service.read_query_profile_page(retried_query_id)
self.__validate_runtime_profiles(retried_profile, handle.get_handle().id)
def __get_retried_query_id_from_profile(self, profile):
"""Returns the entry for 'Retried Query Id' from the given profile, or 'None' if no
such entry exists."""
retried_query_id_search = re.search("Retried Query Id: (.*)", profile)
if not retried_query_id_search: return None
return retried_query_id_search.group(1)
def __wait_until_retried(self, handle, timeout=60):
"""Wait until the given query handle has been retried. This is achieved by polling the
runtime profile of the query and checking the 'Retry Status' field."""
retried_state = "RETRIED"
def __get_retry_status():
profile = self.__get_original_query_profile(handle.get_handle().id)
retry_status = re.search("Retry Status: (.*)", profile)
return retry_status.group(1) if retry_status else None
start_time = time.time()
retry_status = __get_retry_status()
while retry_status != retried_state and time.time() - start_time < timeout:
retry_status = __get_retry_status()
time.sleep(0.5)
if retry_status != retried_state:
raise Timeout("query {0} was not retried within timeout".format
(handle.get_handle().id))
def __kill_random_impalad(self):
"""Kills a random impalad, except for the first node in the cluster, which should be
the Coordinator. Returns the killed impalad."""
killed_impalad = \
self.cluster.impalads[randint(1, ImpalaTestSuite.get_impalad_cluster_size() - 1)]
killed_impalad.kill()
return killed_impalad
def __get_query_id_from_profile(self, profile):
"""Extracts and returns the query id of the given profile."""
query_id_search = re.search("Query \(id=(.*)\)", profile)
assert query_id_search, "Invalid query profile, has no query id"
return query_id_search.group(1)
def __get_original_query_profile(self, original_query_id):
"""Returns the query profile of the original query attempt."""
# TODO (IMPALA-9229): there is no way to get the runtime profiles of the unsuccessful
# query attempts from the ImpalaServer, so fetch them from the debug UI instead.
return self.cluster.get_first_impalad().service.read_query_profile_page(
original_query_id)
def __validate_original_id_in_profile(self, retried_runtime_profile, original_query_id):
"""Validate that the orginal query id is in the 'Original Query Id' entry of the
given retried runtime profile."""
original_id_pattern = "Original Query Id: (.*)"
original_id_search = re.search(original_id_pattern, retried_runtime_profile)
assert original_id_search, \
"Could not find original id pattern '{0}' in profile:\n{1}".format(
original_id_pattern, retried_runtime_profile)
assert original_id_search.group(1) == original_query_id
def __validate_runtime_profiles(self, retried_runtime_profile, original_query_id):
""""Validate the runtime profiles of both the original and retried queries. The
'retried_runtime_profile' refers to the runtime profile of the retried query (the
most recent attempt of the query, which should have succeeded). The
'original_runtime_profile' refers to the runtime profile of the original query (the
original attempt of the query submitted by the user, which failed and had to be
retried)."""
# Extract the retried query id from the retried runtime profile.
retried_query_id = self.__get_query_id_from_profile(retried_runtime_profile)
# Assert that the query id of the original query is in the runtime profile of the
# retried query.
self.__validate_original_id_in_profile(retried_runtime_profile,
original_query_id)
# Get the original runtime profile from the retried runtime profile.
original_runtime_profile = self.__get_original_query_profile(original_query_id)
# Validate the contents of the original runtime profile.
self.__validate_original_runtime_profile(original_runtime_profile, retried_query_id)
# Assert that the query options from the original and retried queries are the same.
assert self.__get_query_options(original_runtime_profile) == \
self.__get_query_options(retried_runtime_profile)
def __get_query_options(self, profile):
"""Returns the query options from the given profile."""
query_options_pattern = "Query Options \(set by configuration and planner\): (.*)"
query_options = re.search(query_options_pattern, profile)
assert query_options, profile
return query_options.group(1)
def __validate_original_runtime_profile(self, original_runtime_profile,
retried_query_id):
"""Validate the contents of the runtime profile of the original query after the query
has been retried."""
# The runtime profile of the original query should reflect that the query failed due
# a retryable error, and that it was retried.
assert "Query State: EXCEPTION" in original_runtime_profile, original_runtime_profile
assert "Impala Query State: ERROR" in original_runtime_profile, \
original_runtime_profile
assert "Query Status: " in original_runtime_profile, \
original_runtime_profile
assert "Retry Status: RETRIED" in original_runtime_profile, original_runtime_profile
assert "Retry Cause: " in original_runtime_profile, \
original_runtime_profile
# Assert that the query id of the retried query is in the runtime profile of the
# original query.
assert "Retried Query Id: {0}".format(retried_query_id) \
in original_runtime_profile, original_runtime_profile
def __validate_web_ui_state(self):
"""Validate the state of the web ui after a query (or queries) have been retried.
The web ui should list 0 queries as in flight, running, or queued."""
impalad_service = self.cluster.get_first_impalad().service
# Assert that the debug web ui shows all queries as completed
self.assert_eventually(60, 0.1,
lambda: impalad_service.get_num_in_flight_queries() == 0)
assert impalad_service.get_num_running_queries('default-pool') == 0
assert impalad_service.get_num_queued_queries('default-pool') == 0
def __assert_executors_blacklisted(self, blacklisted_impalad, profile):
"""Validate that the given profile indicates that the given impalad was blacklisted
during query execution."""
assert "Blacklisted Executors: {0}:{1}".format(blacklisted_impalad.hostname,
blacklisted_impalad.service.be_port) in profile, profile

View File

@@ -0,0 +1,361 @@
#!/usr/bin/env impala-python
#
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
# Runs a stress test for transparent query retries. See the usage of the script for an
# explanation of what the job does and how it works.
# TODO: Add results validation, this likely requires IMPALA-9225 first.
# TODO: Make the script cancellable; more of a nice to have, but Ctrl+C does not kill
# the script, it has to be killed manually (e.g. kill [pid]).
import logging
import pipes
import os
import random
import subprocess
import sys
import threading
import traceback
import Queue
from argparse import ArgumentParser
from argparse import RawDescriptionHelpFormatter
from time import sleep
from tests.common.impala_cluster import ImpalaCluster
from tests.stress.util import create_and_start_daemon_thread
from tests.util.test_file_parser import load_tpc_queries
IMPALA_HOME = os.environ["IMPALA_HOME"]
logging.basicConfig(level=logging.INFO, format='[%(name)s][%(threadName)s]: %(message)s')
LOG = logging.getLogger('query_retries_stress_test')
class QueryRetryLatch(object):
"""
Ensures that the impalad killer thread waits until all queries that are being retried
to complete before killing another impalad. Each thread running a stream of the defined
TPC workload calls 'on_query_completion' whenever it completes a query. The latch then
adds the given stream id to an internal set. The impalad killer thread waits until the
size of the set reaches the total number of concurrent streams before killing another
impalad. The same latch is used multiple times and is reset by the impalad killer
thread each time it kills an impalad.
"""
def __init__(self, num_streams):
self.num_streams = num_streams
self.stream_ids = set()
self.lock = threading.Condition()
def on_query_completion(self, stream_id):
self.lock.acquire()
self.stream_ids.add(stream_id)
if len(self.stream_ids) == self.num_streams:
self.lock.notifyAll()
self.lock.release()
def wait_for_retrying_queries(self):
self.lock.acquire()
while len(self.stream_ids) != self.num_streams:
self.lock.wait()
self.lock.release()
def reset(self):
self.lock.acquire()
self.stream_ids.clear()
self.lock.release()
# All of these parameters need to be global because they are shared amongst threads.
# 'total_queries_retried' is protected by 'total_queries_retried_lock'.
total_queries_retried_lock = threading.Lock()
total_queries_retried = 0
completed_queries_latch = None
def configured_call(cmd):
"""Call a command in a shell with config-impala.sh."""
if type(cmd) is list:
cmd = " ".join([pipes.quote(arg) for arg in cmd])
cmd = "source {0}/bin/impala-config.sh && {1}".format(IMPALA_HOME, cmd)
return subprocess.check_call(["bash", "-c", cmd])
def start_impala_cluster(num_impalads):
"""Start an impalad cluster with 'num_impalads' where there is one exclusive
coordinator and 'num_impalds' - 1 executors."""
configured_call(["{0}/bin/start-impala-cluster.py".format(IMPALA_HOME), "-s",
str(num_impalads), "-c", "1", "--use_exclusive_coordinators"])
def run_concurrent_workloads(concurrency, coordinator, database, queries):
"""Launches 'concurrency' threads, where each thread runs the given set of queries
against the given database in a loop against the given impalad coordinator. The method
waits until all the threads have completed."""
# The exception queue is used to pass errors from the workload threads back to the main
# thread.
exception_queue = Queue.Queue()
# The main method for the workload runner threads.
def __run_workload(stream_id):
global completed_queries_latch
global total_queries_retried_lock
global total_queries_retried
handle = None
num_queries_retried = 0
client = None
try:
# Create and setup the client.
client = coordinator.service.create_beeswax_client()
LOG.info("Running workload: database={0} and coordinator=localhost:{1}, pid={2}"
.format(database, coordinator.get_webserver_port(), coordinator.get_pid()))
client.execute("use {0}".format(database))
client.set_configuration_option('retry_failed_queries', 'true')
# Shuffle the queries in a random order.
shuffled_queries = list(queries.values())
random.shuffle(shuffled_queries)
# Run each query sequentially.
for query in shuffled_queries:
handle = None
try:
# Don't use client.execute as it eagerly fetches results, which causes retries
# to be disabled.
handle = client.execute_async(query)
if not client.wait_for_finished_timeout(handle, 3600):
raise Exception("Timeout while waiting for query to finish")
completed_queries_latch.on_query_completion(stream_id)
# Check if the query was retried, and update any relevant counters.
runtime_profile = client.get_runtime_profile(handle)
if "Original Query Id" in runtime_profile:
LOG.info("Query {0} was retried".format(handle.get_handle().id))
num_queries_retried += 1
total_queries_retried_lock.acquire()
total_queries_retried += 1
total_queries_retried_lock.release()
finally:
if handle:
try:
client.close_query(handle)
except Exception:
pass # suppress any exceptions when closing the query handle
LOG.info("Finished workload, retried {0} queries".format(num_queries_retried))
except Exception:
if handle and handle.get_handle() and handle.get_handle().id:
LOG.exception("Query query_id={0} failed".format(handle.get_handle().id))
exception_queue.put((handle.get_handle().id, sys.exc_info()))
else:
LOG.exception("An unknown query failed")
exception_queue.put(("unknown", sys.exc_info()))
finally:
if client:
client.close()
# Start 'concurrency' number of workload runner threads, and then wait until they all
# complete.
workload_threads = []
LOG.info("Starting {0} concurrent workloads".format(concurrency))
for i in xrange(concurrency):
workload_thread = threading.Thread(target=__run_workload, args=[i],
name="workload_thread_{0}".format(i))
workload_thread.start()
workload_threads.append(workload_thread)
map(lambda thread: thread.join(), workload_threads)
# Check if any of the workload runner threads hit an exception, if one did then print
# the error and exit.
if exception_queue.empty():
LOG.info("All workloads completed")
else:
while not exception_queue.empty():
query_id, exception = exception_queue.get_nowait()
exc_type, exc_value, exc_traceback = exception
LOG.error("A workload failed due to a query failure: query_id={0}\n{1}".format(
query_id, ''.join(traceback.format_exception(
exc_type, exc_value, exc_traceback))))
sys.exit(1)
def start_random_impalad_killer(kill_frequency, start_delay, cluster):
"""Start the impalad killer thread. The thread executes in a constant loop and is
created as a daemon thread so it does not need to complete for the process to
shutdown."""
# The impalad killer thread main method.
def __kill_random_killer():
global completed_queries_latch
while True:
try:
# Pick a random impalad to kill, wait until it is safe to kill the impalad, and
# then kill it.
target_impalad = cluster.impalads[random.randint(1, len(cluster.impalads) - 1)]
sleep(kill_frequency)
completed_queries_latch.wait_for_retrying_queries()
LOG.info("Killing impalad localhost:{0} pid={1}"
.format(target_impalad.get_webserver_port(), target_impalad.get_pid()))
target_impalad.kill()
completed_queries_latch.reset()
# Wait for 'start_delay' seconds before starting the impalad again.
sleep(start_delay)
LOG.info("Starting impalad localhost:{0}"
.format(target_impalad.get_webserver_port()))
target_impalad.start(timeout=300)
except Exception:
LOG.error("Error while running the impalad killer thread", exc_info=True)
# Hard exit the process if the killer thread fails.
sys.exit(1)
# Start the impalad killer thread.
create_and_start_daemon_thread(__kill_random_killer, "impalad_killer_thread")
LOG.info("Started impalad killer with kill frequency {0} and start delay {1}"
.format(kill_frequency, start_delay))
def run_stress_workload(queries, database, workload, start_delay,
kill_frequency, concurrency, iterations, num_impalads):
"""Runs the given set of queries against the the given database. 'concurrency' controls
how many concurrent streams of the queries are run, and 'iterations' controls how many
times the workload is run. 'num_impalads' controls the number of impalads to launch.
The 'kill_frequency' and 'start_delay' are used to configure the impalad killer thread.
'workload' is purely used for debugging purposes."""
# Create the global QueryRetryLatch.
global completed_queries_latch
completed_queries_latch = QueryRetryLatch(concurrency)
# Start the Impala cluster and set the coordinator.
start_impala_cluster(num_impalads)
cluster = ImpalaCluster()
impala_coordinator = cluster.impalads[0]
# Start the 'random impalad killer' thread.
start_random_impalad_killer(kill_frequency, start_delay, cluster)
# Run the stress test 'iterations' times.
for i in xrange(iterations):
LOG.info("Starting iteration {0} of workload {1}".format(i, workload))
run_concurrent_workloads(concurrency, impala_coordinator, database,
queries)
# Print the total number of queries retried.
global total_queries_retried_lock
global total_queries_retried
total_queries_retried_lock.acquire()
LOG.info("Total queries retried {0}".format(total_queries_retried))
total_queries_retried_lock.release()
def parse_args(parser):
"""Parse command line arguments."""
parser.add_argument('-w', '--workload', default='tpch', help="""The target workload to
run. Choices: tpch, tpcds. Default: tpch""")
parser.add_argument('-s', '--scale', default='', help="""The scale factor for the
workload. Default: the scale of the dataload databases - e.g. 'tpch_parquet'""")
parser.add_argument('-t', '--table_format', default='parquet', help="""The file format
to use. Choices: parquet, text. Default: parquet""")
parser.add_argument('-i', '--num_impalads', default='5', help="""The number of impalads
to run. One impalad will be a dedicated coordinator. Default: 5""")
parser.add_argument('-f', '--kill_frequency', default='30', help="""How often, in
seconds, a random impalad should be killed. Default: 30""")
parser.add_argument('-d', '--start_delay', default='10', help="""Number of seconds to
wait before restarting a killed impalad. Default: 10""")
parser.add_argument('-c', '--concurrency', default='4', help="""The number of
concurrent streams of the workload to run. Default: 4""")
parser.add_argument('-r', '--iterations', default='4', help="""The number of
times each workload will be run. Each concurrent stream will execute the workload
this many times. Default: 4""")
args = parser.parse_args()
return args
def main():
# Parse the command line args.
parser = ArgumentParser(description="""
Runs a stress test for transparent query retries. Starts an impala cluster with a
single dedicated coordinator, and a specified number of impalads. Launches multiple
concurrent streams of a TPC workload and randomly kills and starts a single impalad
in the cluster. Only validates that all queries are successful. Prints out a count
of the number of queries retried. A query is considered retried if it has the text
'Original Query Id' in its runtime profile.
The 'iterations' flag controls how many iterations of the TPC workload is run. Each
iteration launches a specified number of concurrent streams of TPC. Each stream runs
all queries in the TPC workload one-by-one, in a random order. A iteration is
considered complete when all concurrent streams successfully finish.
A background thread randomly kills one of the impalads in the cluster, but never
kills the coordinator. The 'kill-frequency' flag controls how often an impalad is
killed, but it is only a lower bound on the actual frequency used. Since query
retries only support retrying a query once, when an impalad is killed, the impalad
killer thread waits until all retried queries complete before killing another
impalad. The 'start-delay' flag controls how long to wait before restarting the
killed impalad. Only one impalad is ever killed at a time.
When specifying a non-default scale, the job will look for a database of the form
'[workload][scale-factor]_parquet' if 'table-format' is parquet or
'[workload][scale-factor] if 'table-format' is text.""",
formatter_class=RawDescriptionHelpFormatter)
args = parse_args(parser)
# Set args to local variables and cast to appropriate types.
scale = args.scale
start_delay = float(args.start_delay)
kill_frequency = float(args.kill_frequency)
concurrency = int(args.concurrency)
iterations = int(args.iterations)
workload = args.workload
table_format = args.table_format
num_impalads = int(args.num_impalads)
# Load TPC queries.
if workload.strip().lower() == 'tpch':
queries = load_tpc_queries('tpch')
elif workload.strip().lower() == 'tpcds':
queries = load_tpc_queries('tpcds')
else:
parser.print_usage()
LOG.error("'--workload' must be either 'tpch' or 'tpcds'")
sys.exit(1)
# Set the correct database.
if table_format is 'parquet':
database = workload + scale + '_parquet'
elif workload is 'text':
database = workload + scale
else:
parser.print_usage()
LOG.info("'--table_format' must be either 'parquet' or 'text'")
sys.exit(1)
# Run the actual stress test.
run_stress_workload(queries, database, workload, start_delay,
kill_frequency, concurrency, iterations, num_impalads)
if __name__ == "__main__":
main()