You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues-all@impala.apache.org by "Riza Suminto (Jira)" <ji...@apache.org> on 2021/03/14 17:18:00 UTC

[jira] [Created] (IMPALA-10584) Investigate intermittent crash in TestScratchLimit::test_with_unlimited_scratch_limit

Riza Suminto created IMPALA-10584:
-------------------------------------

             Summary: Investigate intermittent crash in TestScratchLimit::test_with_unlimited_scratch_limit
                 Key: IMPALA-10584
                 URL: https://issues.apache.org/jira/browse/IMPALA-10584
             Project: IMPALA
          Issue Type: Bug
          Components: Backend
    Affects Versions: Impala 4.0
            Reporter: Riza Suminto
         Attachments: e521ac50-121b-49aa-716f458f-51920619.dmp.resolved.tgz

TestScratchLimit::test_with_unlimited_scratch_limit has been intermittently crashing in ubuntu-16.04-dockerised-tests environment after result spooling is enabled by default in IMPALA-9856. We decide to disable result spooling for TestScratchLimit in IMPALA-10559. However, we still need to investigate what is the root cause of the intermittent crash.

I tried to loop the test overnight with minicluster of 2 impalad and the following modification on test_with_unlimited_scratch_limit query options:
 * spool_query_results=1
 * buffer_pool_limit=24m
 * max_result_spooling_mem=4m

 

At iteration 589, test_with_unlimited_scratch_limit crash with the following logs.

impalad.INFO
{code:java}
I0314 00:15:16.495184 10628 impala-beeswax-server.cc:541] TClientRequest.queryOptions: TQueryOptions {
  01: abort_on_error (bool) = true,
  02: max_errors (i32) = 100,
  03: disable_codegen (bool) = false,
  04: batch_size (i32) = 0,
  05: num_nodes (i32) = 0,
  06: max_scan_range_length (i64) = 0,
  07: num_scanner_threads (i32) = 0,
  11: debug_action (string) = "",
  12: mem_limit (i64) = 0,
  15: hbase_caching (i32) = 0,
  16: hbase_cache_blocks (bool) = false,
  17: parquet_file_size (i64) = 0,
  18: explain_level (i32) = 1,
  19: sync_ddl (bool) = false,
  24: disable_outermost_topn (bool) = false,
  26: query_timeout_s (i32) = 0,
  27: buffer_pool_limit (i64) = 25165824,
  28: appx_count_distinct (bool) = false,
  29: disable_unsafe_spills (bool) = false,
  31: exec_single_node_rows_threshold (i32) = 0,
  32: optimize_partition_key_scans (bool) = false,
  33: replica_preference (i32) = 0,
  34: schedule_random_replica (bool) = false,
  36: disable_streaming_preaggregations (bool) = false,
  37: runtime_filter_mode (i32) = 2,
  38: runtime_bloom_filter_size (i32) = 1048576,
  39: runtime_filter_wait_time_ms (i32) = 0,
  40: disable_row_runtime_filtering (bool) = false,
  41: max_num_runtime_filters (i32) = 10,
  42: parquet_annotate_strings_utf8 (bool) = false,
  43: parquet_fallback_schema_resolution (i32) = 0,
  45: s3_skip_insert_staging (bool) = true,
  46: runtime_filter_min_size (i32) = 1048576,
  47: runtime_filter_max_size (i32) = 16777216,
  48: prefetch_mode (i32) = 1,
  49: strict_mode (bool) = false,
  50: scratch_limit (i64) = -1,
  51: enable_expr_rewrites (bool) = true,
  52: decimal_v2 (bool) = true,
  53: parquet_dictionary_filtering (bool) = true,
  54: parquet_array_resolution (i32) = 0,
  55: parquet_read_statistics (bool) = true,
  56: default_join_distribution_mode (i32) = 0,
  57: disable_codegen_rows_threshold (i32) = 5000,
  58: default_spillable_buffer_size (i64) = 2097152,
  59: min_spillable_buffer_size (i64) = 65536,
  60: max_row_size (i64) = 524288,
  61: idle_session_timeout (i32) = 0,
  62: compute_stats_min_sample_size (i64) = 1073741824,
  63: exec_time_limit_s (i32) = 0,
  64: shuffle_distinct_exprs (bool) = true,
  65: max_mem_estimate_for_admission (i64) = 0,
  66: thread_reservation_limit (i32) = 3000,
  67: thread_reservation_aggregate_limit (i32) = 0,
  68: kudu_read_mode (i32) = 0,
  69: allow_erasure_coded_files (bool) = false,
  70: timezone (string) = "",
  71: scan_bytes_limit (i64) = 0,
  72: cpu_limit_s (i64) = 0,
  73: topn_bytes_limit (i64) = 536870912,
  74: client_identifier (string) = "query_test/test_scratch_limit.py::TestScratchLimit::()::test_with_unlimited_scratch_limit[protocol:beeswax|exec_option:{'batch_size':0;'num_nodes':0;'disable_codegen_rows_threshold':5000;'disable_codegen':False;'abort_on_error':1;'exec_single_node_rows_th",
  75: resource_trace_ratio (double) = 0,
  76: num_remote_executor_candidates (i32) = 3,
  77: num_rows_produced_limit (i64) = 0,
  78: planner_testcase_mode (bool) = false,
  79: default_file_format (i32) = 0,
  80: parquet_timestamp_type (i32) = 0,
  81: parquet_read_page_index (bool) = true,
  82: parquet_write_page_index (bool) = true,
  84: disable_hdfs_num_rows_estimate (bool) = false,
  86: spool_query_results (bool) = true,
  87: default_transactional_type (i32) = 0,
  88: statement_expression_limit (i32) = 250000,
  89: max_statement_length_bytes (i32) = 16777216,
  90: disable_data_cache (bool) = false,
  91: max_result_spooling_mem (i64) = 4194304,
  92: max_spilled_result_spooling_mem (i64) = 1073741824,
  93: disable_hbase_num_rows_estimate (bool) = false,
  94: fetch_rows_timeout_ms (i64) = 10000,
  95: now_string (string) = "",
  96: parquet_object_store_split_size (i64) = 268435456,
  97: mem_limit_executors (i64) = 0,
  98: broadcast_bytes_limit (i64) = 34359738368,
  99: preagg_bytes_limit (i64) = -1,
  100: enable_cnf_rewrites (bool) = true,
  101: max_cnf_exprs (i32) = 200,
  102: kudu_snapshot_read_timestamp_micros (i64) = 0,
  103: retry_failed_queries (bool) = false,
  104: enabled_runtime_filter_types (i32) = 3,
  105: async_codegen (bool) = false,
  106: enable_distinct_semi_join_optimization (bool) = true,
  107: sort_run_bytes_limit (i64) = -1,
  108: max_fs_writers (i32) = 0,
  109: refresh_updated_hms_partitions (bool) = false,
  110: spool_all_results_for_retries (bool) = true,
  112: use_local_tz_for_unix_timestamp_conversions (bool) = false,
  113: convert_legacy_hive_parquet_utc_timestamps (bool) = false,
  114: enable_outer_join_to_inner_transformation (bool) = false,
  115: targeted_kudu_scan_range_length (i64) = -1,
  116: report_skew_limit (double) = 1,
  117: optimize_simple_limit (bool) = false,
  118: use_dop_for_costing (bool) = true,
  119: broadcast_to_partition_factor (double) = 1,
  120: join_rows_produced_limit (i64) = 0,
  121: utf8_mode (bool) = false,
  122: analytic_rank_pushdown_threshold (i64) = 1000,
  123: minmax_filter_threshold (double) = 0,
  124: minmax_filtering_level (i32) = 1,
}
I0314 00:15:16.495311 10628 impala-server.cc:1247] Found local timezone "America/Los_Angeles".
I0314 00:15:16.498564 10628 impala-server.cc:1304] f54227735548245d:fc498f1400000000] Registered query query_id=f54227735548245d:fc498f1400000000 session_id=ec437b1251da7828:462192f6533a37a8
I0314 00:15:16.503772 10628 Frontend.java:1598] f54227735548245d:fc498f1400000000] Analyzing query: select o_orderdate, o_custkey, o_comment
      from tpch.orders
      order by o_orderdate db: default
I0314 00:15:16.504245 10628 BaseAuthorizationChecker.java:110] f54227735548245d:fc498f1400000000] Authorization check took 0 ms
I0314 00:15:16.504266 10628 Frontend.java:1640] f54227735548245d:fc498f1400000000] Analysis and authorization finished.
I0314 00:15:16.506330 10629 admission-controller.cc:1696] f54227735548245d:fc498f1400000000] Trying to admit id=f54227735548245d:fc498f1400000000 in pool_name=default-pool executor_group_name=default per_host_mem_estimate=264.93 MB dedicated_coord_mem_estimate=124.18 MB max_requests=-1 max_queued=200 max_mem=-1.00 B
I0314 00:15:16.506346 10629 admission-controller.cc:1704] f54227735548245d:fc498f1400000000] Stats: agg_num_running=0, agg_num_queued=0, agg_mem_reserved=0,  local_host(local_mem_admitted=0, num_admitted_running=0, num_queued=0, backend_mem_reserved=0, topN_query_stats: queries=[], total_mem_consumed=0; pool_level_stats: num_running=0, min=0, max=0, pool_total_mem=0)
I0314 00:15:16.506363 10629 admission-controller.cc:1212] f54227735548245d:fc498f1400000000] Admitting query id=f54227735548245d:fc498f1400000000
I0314 00:15:16.506400 10629 impala-server.cc:2130] f54227735548245d:fc498f1400000000] Registering query locations
I0314 00:15:16.506425 10629 coordinator.cc:150] f54227735548245d:fc498f1400000000] Exec() query_id=f54227735548245d:fc498f1400000000 stmt=select o_orderdate, o_custkey, o_comment
      from tpch.orders
      order by o_orderdate
I0314 00:15:16.506711 10629 coordinator.cc:474] f54227735548245d:fc498f1400000000] starting execution on 2 backends for query_id=f54227735548245d:fc498f1400000000
I0314 00:15:16.507216 24247 control-service.cc:148] f54227735548245d:fc498f1400000000] ExecQueryFInstances(): query_id=f54227735548245d:fc498f1400000000 coord=lv-desktop:27000 #instances=2
I0314 00:15:16.507802 10633 query-state.cc:948] f54227735548245d:fc498f1400000002] Executing instance. instance_id=f54227735548245d:fc498f1400000002 fragment_idx=1 per_fragment_instance_idx=1 coord_state_idx=0 #in-flight=1
I0314 00:15:16.507810 10629 coordinator.cc:533] f54227735548245d:fc498f1400000000] started execution on 2 backends for query_id=f54227735548245d:fc498f1400000000
I0314 00:15:16.507913 10634 query-state.cc:948] f54227735548245d:fc498f1400000000] Executing instance. instance_id=f54227735548245d:fc498f1400000000 fragment_idx=0 per_fragment_instance_idx=0 coord_state_idx=0 #in-flight=2
I0314 00:15:18.587622 24267 krpc-data-stream-mgr.cc:427] Reduced stream ID cache from 33 items, to 32, eviction took: 0
I0314 00:15:20.481271 10628 impala-beeswax-server.cc:215] get_results_metadata(): query_id=f54227735548245d:fc498f1400000000
F0314 00:15:21.669415 10634 reservation-tracker.cc:436] f54227735548245d:fc498f1400000000] Check failed: used_reservation_.Load() + child_reservations_.Load() <= reservation_.Load() (6291456 vs. 4194304) 4194304 + 2097152 > 4194304
Wrote minidump to /home/riza/workspace/impala/logs/cluster/minidumps/impalad/e521ac50-121b-49aa-716f458f-51920619.dmp
{code}
impalad.ERROR
{code:java}
Log file created at: 2021/03/13 22:41:15
Running on machine: lv-desktop
Log line format: [IWEF]mmdd hh:mm:ss.uuuuuu threadid file:line] msg
E0313 22:41:15.237749 24033 logging.cc:148] stderr will be logged to this file.
21/03/13 22:41:16 INFO util.JvmPauseMonitor: Starting JVM pause monitor
F0314 00:15:21.669415 10634 reservation-tracker.cc:436] f54227735548245d:fc498f1400000000] Check failed: used_reservation_.Load() + child_reservations_.Load() <= reservation_.Load() (6291456 vs. 4194304) 4194304 + 2097152 > 4194304
*** Check failure stack trace: ***
    @          0x549bf3c  google::LogMessage::Fail()
    @          0x549d82c  google::LogMessage::SendToLog()
    @          0x549b89a  google::LogMessage::Flush()
    @          0x549f498  google::LogMessageFatal::~LogMessageFatal()
    @          0x2965d86  impala::ReservationTracker::CheckConsistency()
    @          0x2964ea2  impala::ReservationTracker::TransferReservationTo()
    @          0x29584d3  impala::BufferPool::ClientHandle::SaveReservation()
    @          0x2f31cf2  impala::BufferedTupleStream::NextReadPage()
    @          0x2f33051  impala::BufferedTupleStream::UnpinStream()
    @          0x2fa0575  impala::SpillableRowBatchQueue::AddBatch()
    @          0x2b6f2bc  impala::BufferedPlanRootSink::Send()
    @          0x24cc429  impala::FragmentInstanceState::ExecInternal()
    @          0x24c842f  impala::FragmentInstanceState::Exec()
    @          0x240c567  impala::QueryState::ExecFInstance()
    @          0x240a965  _ZZN6impala10QueryState15StartFInstancesEvENKUlvE_clEv
    @          0x240f536  _ZN5boost6detail8function26void_function_obj_invoker0IZN6impala10QueryState15StartFInstancesEvEUlvE_vE6invokeERNS1_15function_bufferE
    @          0x22b6621  boost::function0<>::operator()()
    @          0x2926cb3  impala::Thread::SuperviseThread()
    @          0x292f614  boost::_bi::list5<>::operator()<>()
    @          0x292f538  boost::_bi::bind_t<>::operator()()
    @          0x292f4f9  boost::detail::thread_data<>::run()
    @          0x41a2f41  thread_proxy
    @     0x7f6d1981f6da  start_thread
    @     0x7f6d160ea71e  clone
Wrote minidump to /home/riza/workspace/impala/logs/cluster/minidumps/impalad/e521ac50-121b-49aa-716f458f-51920619.dmp
{code}
I'm also attaching the resolved dump of e521ac50-121b-49aa-716f458f-51920619.dmp.

It looks like result spooling + sort query + low buffer_pool_limit used in test_with_unlimited_scratch_limit reveal some bug in BufferedPlanRootSink/BufferedTupleStream.
New row batch is added to BufferedPlanRootSink, but there is not enough memory. At the same time, all rows in the read_page_ of the read iterator has been fully read. The underlying BufferedTupleStream try to spill, but I suspect nothing got spilled. Nothing get freed including the read_page_ that has been fully read.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-all-unsubscribe@impala.apache.org
For additional commands, e-mail: issues-all-help@impala.apache.org