Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions rust/datafusion/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ cli = ["rustyline"]

[dependencies]
ahash = "0.6"
hashbrown = "0.9"
arrow = { path = "../arrow", version = "3.0.0-SNAPSHOT", features = ["prettyprint"] }
parquet = { path = "../parquet", version = "3.0.0-SNAPSHOT", features = ["arrow"] }
sqlparser = "0.6.1"
Expand Down
1 change: 0 additions & 1 deletion rust/datafusion/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.

#![warn(missing_docs)]
// Clippy lints, some should be disabled incrementally
#![allow(
Expand Down
28 changes: 15 additions & 13 deletions rust/datafusion/src/physical_plan/hash_aggregate.rs
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ use super::{
SendableRecordBatchStream,
};
use ahash::RandomState;
use std::collections::HashMap;
use hashbrown::HashMap;

use async_trait::async_trait;

Expand Down Expand Up @@ -253,23 +253,25 @@ fn group_aggregate_batch(
// 1.1 construct the key from the group values
// 1.2 construct the mapping key if it does not exist
// 1.3 add the row' index to `indices`

// Make sure we can create the accumulators or otherwise return an error
create_accumulators(aggr_expr).map_err(DataFusionError::into_arrow_external_error)?;

for row in 0..batch.num_rows() {
// 1.1
create_key(&group_values, row, &mut key)
.map_err(DataFusionError::into_arrow_external_error)?;

match accumulators.get_mut(&key) {
// 1.2
None => {
let accumulator_set = create_accumulators(aggr_expr)
.map_err(DataFusionError::into_arrow_external_error)?;

accumulators
.insert(key.clone(), (accumulator_set, Box::new(vec![row as u32])));
}
accumulators
.raw_entry_mut()
.from_key(&key)
// 1.3
Some((_, v)) => v.push(row as u32),
}
.and_modify(|_, (_, v)| v.push(row as u32))
// 1.2
.or_insert_with(|| {
// We can safely unwrap here as we checked we can create an accumulator before
let accumulator_set = create_accumulators(aggr_expr).unwrap();
(key.clone(), (accumulator_set, Box::new(vec![row as u32])))
});
}

// 2.1 for each key
Expand Down
17 changes: 7 additions & 10 deletions rust/datafusion/src/physical_plan/hash_join.rs
Original file line number Diff line number Diff line change
Expand Up @@ -19,13 +19,11 @@
//! into a set of partitions.

use std::sync::Arc;
use std::{
any::Any,
collections::{HashMap, HashSet},
};
use std::{any::Any, collections::HashSet};

use async_trait::async_trait;
use futures::{Stream, StreamExt, TryStreamExt};
use hashbrown::HashMap;

use arrow::array::{make_array, Array, MutableArrayData};
use arrow::datatypes::{Schema, SchemaRef};
Expand Down Expand Up @@ -214,12 +212,11 @@ fn update_hash(
// update the hash map
for row in 0..batch.num_rows() {
create_key(&keys_values, row, &mut key)?;
match hash.get_mut(&key) {
Some(v) => v.push((index, row)),
None => {
hash.insert(key.clone(), vec![(index, row)]);
}
};

hash.raw_entry_mut()
.from_key(&key)
.and_modify(|_, v| v.push((index, row)))
.or_insert_with(|| (key.clone(), vec![(index, row)]));
}
Ok(())
}
Expand Down