The query retry launches in a separate thread. This thread may not
finishes when deleting the query from the given QueryDriverMap if
the query retry was failed launched. In this case, the resources
for the query retry thread will not release. So the reference
count of QueryDriver (via the shared_ptr) will not go to 0 and it
will not be destroyed.
We need wait until the query retry thread execution has completed
when deleting the query from the given QueryDiverMap.
Testing:
Modify the test_query_retries.py to verify memory leak by checking
the debug web UI of memz.
Change-Id: If804ca65da1794c819a6b2e6567ea7651ab5112f
Reviewed-on: http://gerrit.cloudera.org:8080/17735
Reviewed-by: Wenzhe Zhou <wzhou@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Reviewed-by: Quanlong Huang <huangquanlong@gmail.com>
This patch provides count(star) optimization for ORC scans, similar to
the work done in IMPALA-5036 for Parquet scans. We use the stripes num
rows statistics when computing the count star instead of materializing
empty rows. The aggregate function changed from a count to a special sum
function initialized to 0.
This count(star) optimization is disabled for the full ACID table
because the scanner might need to read and validate the
'currentTransaction' column in table's special schema.
This patch drops 'parquet' from names related to the count star
optimization. It also improves the count(star) operation in general by
serving the result just from the file's footer stats for both Parquet
and ORC. We unify the optimized count star and zero slot scan functions
into HdfsColumnarScanner.
The following table shows a performance comparison before and after the
patch. primitive_count_star query target tpch10_parquet.lineitem
table (10GB scale TPC-H). Meanwhile, count_star_parq and count_star_orc
query is a modified primitive_count_star query that targets
tpch_parquet.lineitem and tpch_orc_def.lineitem table accordingly.
+-------------------+----------------------+-----------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+-------+
| Workload | Query | File Format | Avg(s) | Base Avg(s) | Delta(Avg) | StdDev(%) | Base StdDev(%) | Iters | Median Diff(%) | MW Zval | Tval |
+-------------------+----------------------+-----------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+-------+
| tpch_parquet | count_star_parq | parquet / none / none | 0.06 | 0.07 | -10.45% | 2.87% | * 25.51% * | 9 | -1.47% | -1.26 | -1.22 |
| tpch_orc_def | count_star_orc | orc / def / none | 0.06 | 0.08 | -22.37% | 6.22% | * 30.95% * | 9 | -1.85% | -1.16 | -2.14 |
| TARGETED-PERF(10) | primitive_count_star | parquet / none / none | 0.06 | 0.08 | I -30.40% | 2.68% | * 29.63% * | 9 | I -7.20% | -2.42 | -3.07 |
+-------------------+----------------------+-----------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+-------+
Testing:
- Add PlannerTest.testOrcStatsAgg
- Add TestAggregationQueries::test_orc_count_star_optimization
- Exercise count(star) in TestOrc::test_misaligned_orc_stripes
- Pass core tests
Change-Id: I0fafa1182f97323aeb9ee39dd4e8ecd418fa6091
Reviewed-on: http://gerrit.cloudera.org:8080/18327
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
This patch adds support for COS(Cloud Object Storage). Using the
hadoop-cos, the implementation is similar to other remote FileSystems.
New flags for COS:
- num_cos_io_threads: Number of COS I/O threads. Defaults to be 16.
Follow-up:
- Support for caching COS file handles will be addressed in
IMPALA-10772.
- test_concurrent_inserts and test_failing_inserts in
test_acid_stress.py are skipped due to slow file listing on
COS (IMPALA-10773).
Tests:
- Upload hdfs test data to a COS bucket. Modify all locations in HMS
DB to point to the COS bucket. Remove some hdfs caching params.
Run CORE tests.
Change-Id: Idce135a7591d1b4c74425e365525be3086a39821
Reviewed-on: http://gerrit.cloudera.org:8080/17503
Reviewed-by: Joe McDonnell <joemcdonnell@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
The crash happens when canceling the retrying query in web UI.
The canceling action will call ImpalaServer#UnregisterQuery.
The QueryDriver will be null if the query has already been unregistered.
Testing:
Add test in tests/custom_cluster/test_query_retries.py and manually
tested 100 times to make sure that there was no Impalad crash
Change-Id: I3b9a2cccbfbdca00b099e0f8d5f2d4bcb4d0a8c3
Reviewed-on: http://gerrit.cloudera.org:8080/17729
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
When query retry fails in RetryQueryFromThread(), the retried query id
may not be unregistered if the failure happens before we store the
retry_request_state. In this case, QueryDriver::Unregister() has no way
to get the retried query id so it's not deleted. Note that the retried
query id is registered in RetryQueryFromThread() so should be deleted
later. This finally results in a leak in the query driver map, where
queries in it are shown as in-flight queries.
test_retry_query_result_cacheing_failed and
test_retry_query_set_query_in_flight_failed (added in IMPALA-10413)
asserts one in-flight query at the end. This is satisfied by the leak.
Instead, we should verify no running queries at the end.
This patch adds a new field in QueryDriver to remember the registered
retry query id as a backup way for getting it when query retry fails
before we store the ClientRequestState of the retried query (so
retried_client_request_state_ is null).
Tests:
- Run test_retry_query_result_cacheing_failed and
test_retry_query_set_query_in_flight_failed 100 times.
Change-Id: I074526799d68041a425b2379e74f8d8b45ce892a
Reviewed-on: http://gerrit.cloudera.org:8080/17465
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
The crash happens when canceling the retrying query.
If the original query was unregistered while the new query was
being created, it will call HandleRetryFailure to abort the new
query. But the status is ok, so when calling Status::AddDetail
impalad will crash.
After the WaitAsync interface called and before the
retry_request_state moved to retried_client_request_state_ , if
abort the new retry query, retry_request_state need to call
Finalize, otherwise the wait-thread will leak.
In some cases like canceled the original query or closed the session
we may not create the new query, so we also check whether the query
is retried.
Tests:
Add test in tests/custom_cluster/test_query_retries.py and manually
tested 100 times to make sure that there was no Impalad crash
Change-Id: I4fd7228acd0a70d33859029052239f9b9f795e5d
Reviewed-on: http://gerrit.cloudera.org:8080/16911
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
This patch adds support for GCS(Google Cloud Storage). Using the
gcs-connector, the implementation is similar to other remote
FileSystems.
New flags for GCS:
- num_gcs_io_threads: Number of GCS I/O threads. Defaults to be 16.
Follow-up:
- Support for spilling to GCS will be addressed in IMPALA-10561.
- Support for caching GCS file handles will be addressed in
IMPALA-10568.
- test_concurrent_inserts and test_failing_inserts in
test_acid_stress.py are skipped due to slow file listing on
GCS (IMPALA-10562).
- Some tests are skipped due to issues introduced by /etc/hosts setting
on GCE instances (IMPALA-10563).
Tests:
- Compile and create hdfs test data on a GCE instance. Upload test data
to a GCS bucket. Modify all locations in HMS DB to point to the GCS
bucket. Remove some hdfs caching params. Run CORE tests.
- Compile and load snapshot data to a GCS bucket. Run CORE tests.
Change-Id: Ia91ec956de3b620cccf6a1244b56b7da7a45b32b
Reviewed-on: http://gerrit.cloudera.org:8080/17121
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Result spooling has been relatively stable since it was introduced, and
it has several benefits described in IMPALA-8656. This patch enable
result spooling (SPOOL_QUERY_RESULTS) query options by default.
Furthermore, some tests need to be adjusted to account for result
spooling by default. The following are the adjustment categories and
list of tests that fall under such category.
Change in assertions:
PlannerTest#testAcidTableScans
PlannerTest#testBloomFilterAssignment
PlannerTest#testConstantFolding
PlannerTest#testFkPkJoinDetection
PlannerTest#testFkPkJoinDetectionWithHDFSNumRowsEstDisabled
PlannerTest#testKuduSelectivity
PlannerTest#testMaxRowSize
PlannerTest#testMinMaxRuntimeFilters
PlannerTest#testMinMaxRuntimeFiltersWithHDFSNumRowsEstDisabled
PlannerTest#testMtDopValidation
PlannerTest#testParquetFiltering
PlannerTest#testParquetFilteringDisabled
PlannerTest#testPartitionPruning
PlannerTest#testPreaggBytesLimit
PlannerTest#testResourceRequirements
PlannerTest#testRuntimeFilterQueryOptions
PlannerTest#testSortExprMaterialization
PlannerTest#testSpillableBufferSizing
PlannerTest#testTableSample
PlannerTest#testTpch
PlannerTest#testKuduTpch
PlannerTest#testTpchNested
PlannerTest#testUnion
TpcdsPlannerTest
custom_cluster/test_admission_controller.py::TestAdmissionController::test_dedicated_coordinator_planner_estimates
custom_cluster/test_admission_controller.py::TestAdmissionController::test_memory_rejection
custom_cluster/test_admission_controller.py::TestAdmissionController::test_pool_mem_limit_configs
metadata/test_explain.py::TestExplain::test_explain_level2
metadata/test_explain.py::TestExplain::test_explain_level3
metadata/test_stats_extrapolation.py::TestStatsExtrapolation::test_stats_extrapolation
Increase BUFFER_POOL_LIMIT:
query_test/test_queries.py::TestQueries::test_analytic_fns
query_test/test_runtime_filters.py::TestRuntimeRowFilters::test_row_filter_reservation
query_test/test_sort.py::TestQueryFullSort::test_multiple_mem_limits_full_output
query_test/test_spilling.py::TestSpillingBroadcastJoins::test_spilling_broadcast_joins
query_test/test_spilling.py::TestSpillingDebugActionDimensions::test_spilling_aggs
query_test/test_spilling.py::TestSpillingDebugActionDimensions::test_spilling_regression_exhaustive
query_test/test_udfs.py::TestUdfExecution::test_mem_limits
Increase MEM_LIMIT:
query_test/test_mem_usage_scaling.py::TestExchangeMemUsage::test_exchange_mem_usage_scaling
query_test/test_mem_usage_scaling.py::TestScanMemLimit::test_hdfs_scanner_thread_mem_scaling
Increase MAX_ROW_SIZE:
custom_cluster/test_parquet_max_page_header.py::TestParquetMaxPageHeader::test_large_page_header_config
query_test/test_insert.py::TestInsertQueries::test_insert_large_string
query_test/test_query_mem_limit.py::TestQueryMemLimit::test_mem_limit
query_test/test_scanners.py::TestTextSplitDelimiters::test_text_split_across_buffers_delimiter
query_test/test_scanners.py::TestWideRow::test_wide_row
Disable result spooling to maintain assertion:
custom_cluster/test_admission_controller.py::TestAdmissionController::test_set_request_pool
custom_cluster/test_admission_controller.py::TestAdmissionController::test_timeout_reason_host_memory
custom_cluster/test_admission_controller.py::TestAdmissionController::test_timeout_reason_pool_memory
custom_cluster/test_admission_controller.py::TestAdmissionController::test_queue_reasons_memory
custom_cluster/test_admission_controller.py::TestAdmissionController::test_pool_config_change_while_queued
custom_cluster/test_query_retries.py::TestQueryRetries::test_retry_fetched_rows
custom_cluster/test_query_retries.py::TestQueryRetries::test_retry_finished_query
custom_cluster/test_scratch_disk.py::TestScratchDir::test_no_dirs
custom_cluster/test_scratch_disk.py::TestScratchDir::test_non_existing_dirs
custom_cluster/test_scratch_disk.py::TestScratchDir::test_non_writable_dirs
query_test/test_insert.py::TestInsertQueries::test_insert_large_string (the last query only)
query_test/test_kudu.py::TestKuduMemLimits::test_low_mem_limit_low_selectivity_scan
query_test/test_mem_usage_scaling.py::TestScanMemLimit::test_kudu_scan_mem_usage
query_test/test_queries.py::TestQueriesParquetTables::test_very_large_strings
query_test/test_query_mem_limit.py::TestCodegenMemLimit::test_codegen_mem_limit
shell/test_shell_client.py::TestShellClient::test_fetch_size
Testing:
- Pass exhaustive tests.
Change-Id: I9e360c1428676d8f3fab5d95efee18aca085eba4
Reviewed-on: http://gerrit.cloudera.org:8080/16755
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
This patch extends blacklist functionality by adding executor node to
blacklist if a query fails caused by disk failure during spill-to-disk.
Also classifies disk error codes and defines a blacklistable error set
for non-transient disk errors. Coordinator blacklists executor only if
the executor hitted blacklistable error during spill-to-disk.
Adds a new debug action to simulate disk write error during spill-to-
disk. To use, specify in query options as:
'debug_action': 'IMPALA_TMP_FILE_WRITE:<hostname>:<port>:<action>'
where <hostname> and <port> represent the impalad which execute the
fragment instances, <port> is the BE krpc port (default 27000).
Adds new test cases for blacklist and query-retry to cover the code
changes.
Testing:
- Passed new test cases.
- Passed exhaustive test.
- Manually simulated disk failures in scratch directories on nodes
of a cluster, verified that the nodes were blacklisted as
expected.
Change-Id: I04bfcb7f2e0b1ef24a5b4350f270feecd8c47437
Reviewed-on: http://gerrit.cloudera.org:8080/16949
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
PlanRootSink can fail silently if result spooling is enabled and
maxMemReservationBytes is less than 2 * MAX_ROW_SIZE. This happens
because results are spilled using a SpillableRowBatchQueue which needs 2
buffer (read and write) with at least MAX_ROW_SIZE bytes per buffer.
This patch fixes this by setting a lower bound of 2 * MAX_ROW_SIZE while
computing the min reservation for the PlanRootSink.
Testing:
- Pass exhaustive tests.
- Add e2e TestResultSpoolingMaxReservation.
- Lower MAX_ROW_SIZE on tests where MAX_RESULT_SPOOLING_MEM is set to
extremely low value. Also verify that PLAN_ROOT_SINK's ReservationLimit
remain unchanged after lowering the MAX_ROW_SIZE.
Change-Id: Id7138e1e034ea5d1cd15cf8de399690e52a9d726
Reviewed-on: http://gerrit.cloudera.org:8080/16765
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
The legacy Thrift based Impala internal service has been removed so
the backend port 22000 can be freed up.
This patch set flag be_port as a REMOVED_FLAG and all infrastructures
around it are cleaned up. StatestoreSubscriber::subscriber_id is set
as hostname + krpc_port.
Testing:
- Passed the exhaustive test.
Change-Id: Ic6909a8da449b4d25ee98037b3eb459af4850dc6
Reviewed-on: http://gerrit.cloudera.org:8080/16533
Reviewed-by: Thomas Tauber-Marshall <tmarshall@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
The legacy Thrift based Impala internal service has been deprecated
and can be removed now.
This patch removes ImpalaInternalService. All infrastructures around it
are cleaned up, except one place for flag be_port.
StatestoreSubscriber::subscriber_id consists be_port, but we cannot
change format of subscriber_id now. This remaining be_port issue will be
fixed in a succeeding patch (part 4).
TQueryCtx.coord_address is changed to TQueryCtx.coord_hostname since the
port in TQueryCtx.coord_address is set as be_port and is unused now.
Also Rename TQueryCtx.coord_krpc_address as TQueryCtx.coord_ip_address.
Testing:
- Passed the exhaustive test.
- Passed Quasar-L0 test.
Change-Id: I5fa83c8009590124dded4783f77ef70fa30119e6
Reviewed-on: http://gerrit.cloudera.org:8080/16291
Reviewed-by: Thomas Tauber-Marshall <tmarshall@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
This patch contains the following refactors that are needed for the
admission control service, in order to make the main patch easier to
review:
- Adds a new class AdmissionControlClient which will be used to
abstract the logic for submitting queries to either a local or
remote admission controller out from ClientRequestState/Coordinator.
Currently only local submission is supported.
- SubmitForAdmission now takes a BackendId representing the
coordinator instead of assuming that the local impalad will be the
coordinator.
- The CRS_BEFORE_ADMISSION debug action is moved into
SubmitForAdmission() so that it will be executed on whichever daemon
is performing admission control rather than always on the
coordinator (needed for TestAdmissionController.test_cancellation).
- ShardedQueryMap is extended to allow keys to be either TUniqueId or
UniqueIdPB and Add(), Get(), and Delete() convenience functions are
added.
- Some utils related to seralizing Thrift objects into sidecars are
added.
Testing:
- Passed a run of existing core tests.
Change-Id: I7974a979cf05ed569f31e1ab20694e29fd3e4508
Reviewed-on: http://gerrit.cloudera.org:8080/16411
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
When a node is blacklisted, it is only placed on the blacklist for a
certain period of time. For the current implementation, it is possible
that the retried query could end up running on the node that it
blacklisted during its original attempt. To avoid same failure for
the retried query, we should not schedule query fragment instances on
the blacklisted nodes which caused the original query to fail.
This patch filters out the executors from executor group for those
nodes which are blacklisted during its original attempt when make
schedule for the retried query.
Adds new test cases test_retry_exec_rpc_failure_before_admin_delay()
and test_retry_query_failure_all_executors_blacklisted() for retried
queries which are triggered by RPC failure and blacklist timeout
are triggered by adding delay before admission.
Testing:
- Passed test_query_retries.py, including the new test cases.
- Passed core tests.
Change-Id: I00bc1b5026efbd0670ffbe57bcebc457d34cb105
Reviewed-on: http://gerrit.cloudera.org:8080/16369
Reviewed-by: Sahil Takiar <stakiar@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
A query will come into the FINISHED state when some rows are available,
even when some fragment instances are still executing. When a retryable
query comes into the FINISHED state and the client hasn't fetched any
results, we are still able to retry it for any retryable failures. This
patch fixes a DCHECK when retrying a FINISHED state query.
Tests:
- Add a test in test_query_retries.py for retrying a query in FINISHED
state.
Change-Id: I11d82bf80640760a47325833463def8a3791bdda
Reviewed-on: http://gerrit.cloudera.org:8080/16351
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
If we have returned any results to the client in the original query,
query retry will be skipped to avoid incorrect results. This patch adds
a query option, spool_all_results_for_retries, for retryable queries to
spool all results before returning any to the client. It defaults to
true. If all query results cannot be contained in the allocated result
spooling space, we'll return results and thus disabled query retry on
the query.
Setting spool_all_results_for_retries to false will fallback to the
original behavior - client can fetch results when any of them are ready.
So we explicitly set it to false in the retried query since it won't be
retried. For non retryable queries or queries that don't enable results
spooling, the spool_all_results_for_retries option takes no effect.
To implement this, this patch defers the time when results are ready to
be fetched. By default, the “rows available” event happens when any
results are ready. For a retryable query, when spool_query_results and
spool_all_results_for_retries are both true, the “rows available” event
happens after all results are spooled or any errors stopping us to do
so, e.g. batch queue is full, cancellation or failures. After waiting
for the root fragment instance’s Open() finishes, the coordinator will
wait until results of BufferedPlanRootSink are ready.
BufferedPlanRootSink sets the results ready signal in its Send(),
Close(), Cancel(), FlushFinal() methods.
Tests:
- Add a test to verify that a retryable query will spool all its results
when results spooling and spool_all_results_for_retries are enabled.
- Add a test to verify that query retry succeeds when a retryable query
is still spooling its results (spool_all_results_for_retries=true).
- Add a test to verify that the retried query won't spool all results
even when results spooling and spool_all_results_for_retries are
enabled in the original query.
- Add a test to verify that the original query can be canceled
correctly. We need this because the added logics for
spool_all_results_for_retries are related to the cancellation code
path.
- Add a test to verify results will be returned when all of them can't
fit into the result spooling space, and query retry will be skipped.
Change-Id: I462dbfef9ddab9060b30a6937fca9122484a24a5
Reviewed-on: http://gerrit.cloudera.org:8080/16323
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
This patch skips all tests in TestQueryRetries on EC builds.
The tests in TestQueryRetries runs queries that run on three instances
during regular builds (HDFS, S3, etc.), but only two instances on EC
builds. This causes some non-deterministism during the test because
killing an impalad in the mini-cluster won't necessarily cause a retry
to be triggered.
It bumps up the timeout used when waiting for a query to be retried.
It improves the assertion in __get_query_id_from_profile so that it
dumps the full profile when the assertion fails. This should help
debuggability of any test failures that fail in this assertion.
Testing:
* Ran TestQueryRetries locally
Change-Id: Id5c73c2cbd0ef369175856c41f36d4b0de4b8d71
Reviewed-on: http://gerrit.cloudera.org:8080/16149
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Impala-shell periodically calls GetExecSummary() when the query is
queuing or running. If the query is being retried, GetExecSummary()
should return the TExecSummary of the retried query. So the progress bar
and live_summary can reflect the most recent state.
This patch also modifies get_summary() to return retry information in
error_logs of TExecSummary. Impala-shell and other clients can print the
info right after the query starts being retried. Modified impala-shell
to print the retried query link when the retried query is running.
Example output when the retried query is running:
Query: select count(*) from functional.alltypes where bool_col = sleep(60)
Query submitted at: 2020-06-18 22:08:49 (Coordinator: http://quanlong-OptiPlex-BJ:25000)
Query progress can be monitored at: http://quanlong-OptiPlex-BJ:25000/query_plan?query_id=9444fe7f0df0da28:29134b0800000000
Failed due to unreachable impalad(s): quanlong-OptiPlex-BJ:22001
Retrying query using query id: 5748d9a3ccc28ba8:a75e2fab00000000
Retried query link: http://quanlong-OptiPlex-BJ:25000/query_plan?query_id=5748d9a3ccc28ba8:a75e2fab00000000
[############################### ] 50%
Tests:
- Manually verify the progress bar and live_summary work when the query
is being retried.
- Add tests in test_query_retries.py to validate the get_summary()
results.
Change-Id: I8f96919f00e0b64d589efd15b6b5ec82fb725d56
Reviewed-on: http://gerrit.cloudera.org:8080/16096
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Beeswax clients use get_log() to retrieve the warning/error message
after the query finishes. HS2 clients use GetLog() for the same purpose.
This patch adds the retry information into the returned result if the
query is retried. So clients that print the log can show the original
query failure and the retried query id.
This patch also modifies impala-shell to extract the retried query id
and print the retried query link.
Here's an example of the impala-shell output:
Query: select count(*) from functional.alltypes where bool_col = sleep(60)
Query submitted at: 2020-06-18 21:23:52 (Coordinator: http://quanlong-OptiPlex-BJ:25000)
Query progress can be monitored at: http://quanlong-OptiPlex-BJ:25000/query_plan?query_id=7944ffee4d81cdd4:e7f9357a00000000
+----------+
| count(*) |
+----------+
| 3650 |
+----------+
WARNINGS: Original query failed:
Failed due to unreachable impalad(s): quanlong-OptiPlex-BJ:22001
Query has been retried using query id: 934b2734f67a1161:a0dbd60200000000
Retried query link: http://quanlong-OptiPlex-BJ:25000/query_plan?query_id=934b2734f67a1161:a0dbd60200000000
Tests:
- Add tests in test_query_retries.py to verify client logs returned
from GetLog().
- Run test_query_retries.py.
- Manually run queries in impala-shell and kill impalads. Verify
printed messages when the retried queries succeed or fail.
Change-Id: I58cf94f91a0b92eb9a3088bee3894ac157a954dc
Reviewed-on: http://gerrit.cloudera.org:8080/16093
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
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>