|
| 1 | +// Licensed to the Apache Software Foundation (ASF) under one |
| 2 | +// or more contributor license agreements. See the NOTICE file |
| 3 | +// distributed with this work for additional information |
| 4 | +// regarding copyright ownership. The ASF licenses this file |
| 5 | +// to you under the Apache License, Version 2.0 (the |
| 6 | +// "License"); you may not use this file except in compliance |
| 7 | +// with the License. You may obtain a copy of the License at |
| 8 | +// |
| 9 | +// http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | +// |
| 11 | +// Unless required by applicable law or agreed to in writing, |
| 12 | +// software distributed under the License is distributed on an |
| 13 | +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
| 14 | +// KIND, either express or implied. See the License for the |
| 15 | +// specific language governing permissions and limitations |
| 16 | +// under the License. |
| 17 | + |
| 18 | +//! This module contains tests for limiting memory at runtime in DataFusion |
| 19 | +
|
| 20 | +use std::sync::Arc; |
| 21 | + |
| 22 | +use arrow::record_batch::RecordBatch; |
| 23 | +use datafusion::datasource::MemTable; |
| 24 | +use datafusion::execution::disk_manager::DiskManagerConfig; |
| 25 | +use datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv}; |
| 26 | +use datafusion_common::assert_contains; |
| 27 | + |
| 28 | +use datafusion::prelude::{SessionConfig, SessionContext}; |
| 29 | +use test_utils::{stagger_batch, AccessLogGenerator}; |
| 30 | + |
| 31 | +#[cfg(test)] |
| 32 | +#[ctor::ctor] |
| 33 | +fn init() { |
| 34 | + let _ = env_logger::try_init(); |
| 35 | +} |
| 36 | + |
| 37 | +#[tokio::test] |
| 38 | +async fn oom_sort() { |
| 39 | + run_limit_test( |
| 40 | + "select * from t order by host DESC", |
| 41 | + "Resources exhausted: Memory Exhausted while Sorting (DiskManager is disabled)", |
| 42 | + ) |
| 43 | + .await |
| 44 | +} |
| 45 | + |
| 46 | +#[tokio::test] |
| 47 | +async fn group_by_none() { |
| 48 | + run_limit_test( |
| 49 | + "select median(image) from t", |
| 50 | + "Resources exhausted: Cannot spill AggregationState", |
| 51 | + ) |
| 52 | + .await |
| 53 | +} |
| 54 | + |
| 55 | +#[tokio::test] |
| 56 | +async fn group_by_row_hash() { |
| 57 | + run_limit_test( |
| 58 | + "select count(*) from t GROUP BY response_bytes", |
| 59 | + "Resources exhausted: Cannot spill GroupBy Hash (Row) AggregationState", |
| 60 | + ) |
| 61 | + .await |
| 62 | +} |
| 63 | + |
| 64 | +#[tokio::test] |
| 65 | +async fn group_by_hash() { |
| 66 | + run_limit_test( |
| 67 | + // group by dict column |
| 68 | + "select count(*) from t GROUP BY service, host, pod, container", |
| 69 | + "Resources exhausted: Cannot spill GroupBy Hash Accumulators", |
| 70 | + ) |
| 71 | + .await |
| 72 | +} |
| 73 | + |
| 74 | +/// 100K memory limit |
| 75 | +const MEMORY_LIMIT_BYTES: usize = 50; |
| 76 | +const MEMORY_FRACTION: f64 = 0.95; |
| 77 | + |
| 78 | +/// runs the specified query against 1000 rows with a 50 |
| 79 | +/// byte memory limit and no disk manager enabled. |
| 80 | +async fn run_limit_test(query: &str, expected_error: &str) { |
| 81 | + let generator = AccessLogGenerator::new().with_row_limit(Some(1000)); |
| 82 | + |
| 83 | + let batches: Vec<RecordBatch> = generator |
| 84 | + // split up into more than one batch, as the size limit in sort is not enforced until the second batch |
| 85 | + .flat_map(stagger_batch) |
| 86 | + .collect(); |
| 87 | + |
| 88 | + let table = MemTable::try_new(batches[0].schema(), vec![batches]).unwrap(); |
| 89 | + |
| 90 | + let rt_config = RuntimeConfig::new() |
| 91 | + // do not allow spilling |
| 92 | + .with_disk_manager(DiskManagerConfig::Disabled) |
| 93 | + // Only allow 50 bytes |
| 94 | + .with_memory_limit(MEMORY_LIMIT_BYTES, MEMORY_FRACTION); |
| 95 | + |
| 96 | + let runtime = RuntimeEnv::new(rt_config).unwrap(); |
| 97 | + |
| 98 | + let ctx = SessionContext::with_config_rt(SessionConfig::new(), Arc::new(runtime)); |
| 99 | + ctx.register_table("t", Arc::new(table)) |
| 100 | + .expect("registering table"); |
| 101 | + |
| 102 | + let df = ctx.sql(query).await.expect("Planning query"); |
| 103 | + |
| 104 | + match df.collect().await { |
| 105 | + Ok(_batches) => { |
| 106 | + panic!("Unexpected success when running, expected memory limit failure") |
| 107 | + } |
| 108 | + Err(e) => { |
| 109 | + assert_contains!(e.to_string(), expected_error); |
| 110 | + } |
| 111 | + } |
| 112 | +} |
0 commit comments