mirror of
https://github.com/apache/impala.git
synced 2025-12-25 02:03:09 -05:00
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:
committed by
Sahil Takiar
parent
fcf08d1822
commit
bd4d01a379
@@ -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
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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) {
|
||||
|
||||
@@ -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();
|
||||
|
||||
373
be/src/runtime/query-driver.cc
Normal file
373
be/src/runtime/query-driver.cc
Normal 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);
|
||||
}
|
||||
}
|
||||
259
be/src/runtime/query-driver.h
Normal file
259
be/src/runtime/query-driver.h
Normal 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};
|
||||
};
|
||||
}
|
||||
@@ -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
|
||||
)
|
||||
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
@@ -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);
|
||||
}
|
||||
|
||||
|
||||
@@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -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),
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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, ®istered_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, ®istered_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 {
|
||||
|
||||
@@ -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);
|
||||
|
||||
@@ -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()) {
|
||||
44
be/src/service/query-driver-map.h
Normal file
44
be/src/service/query-driver-map.h
Normal 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);
|
||||
};
|
||||
}
|
||||
@@ -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: {
|
||||
|
||||
@@ -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.
|
||||
|
||||
@@ -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;
|
||||
|
||||
@@ -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_;
|
||||
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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.
|
||||
|
||||
@@ -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,
|
||||
|
||||
@@ -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."""
|
||||
|
||||
563
tests/custom_cluster/test_query_retries.py
Normal file
563
tests/custom_cluster/test_query_retries.py
Normal 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
|
||||
361
tests/stress/query_retries_stress_runner.py
Executable file
361
tests/stress/query_retries_stress_runner.py
Executable 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()
|
||||
Reference in New Issue
Block a user