forked from apache/datafusion
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathnyctaxi.rs
156 lines (136 loc) · 5.43 KB
/
nyctaxi.rs
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
// 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.
//! Apache Arrow Rust Benchmarks
use std::collections::HashMap;
use std::path::PathBuf;
use std::process;
use std::time::Instant;
use datafusion::arrow::datatypes::{DataType, Field, Schema};
use datafusion::arrow::util::pretty;
use datafusion::error::Result;
use datafusion::execution::context::{SessionConfig, SessionContext};
use datafusion::physical_plan::collect;
use datafusion::prelude::{CsvReadOptions, ParquetReadOptions};
use structopt::StructOpt;
#[cfg(feature = "snmalloc")]
#[global_allocator]
static ALLOC: snmalloc_rs::SnMalloc = snmalloc_rs::SnMalloc;
#[derive(Debug, StructOpt)]
#[structopt(name = "Benchmarks", about = "Apache Arrow Rust Benchmarks.")]
struct Opt {
/// Activate debug mode to see query results
#[structopt(short, long)]
debug: bool,
/// Number of iterations of each test run
#[structopt(short = "i", long = "iterations", default_value = "3")]
iterations: usize,
/// Number of partitions to process in parallel
#[structopt(short = "p", long = "partitions", default_value = "2")]
partitions: usize,
/// Batch size when reading CSV or Parquet files
#[structopt(short = "s", long = "batch-size", default_value = "8192")]
batch_size: usize,
/// Path to data files
#[structopt(parse(from_os_str), required = true, short = "p", long = "path")]
path: PathBuf,
/// File format: `csv` or `parquet`
#[structopt(short = "f", long = "format", default_value = "csv")]
file_format: String,
}
#[tokio::main]
async fn main() -> Result<()> {
let opt = Opt::from_args();
println!("Running benchmarks with the following options: {:?}", opt);
let config = SessionConfig::new()
.with_target_partitions(opt.partitions)
.with_batch_size(opt.batch_size);
let mut ctx = SessionContext::with_config(config);
let path = opt.path.to_str().unwrap();
match opt.file_format.as_str() {
"csv" => {
let schema = nyctaxi_schema();
let options = CsvReadOptions::new().schema(&schema).has_header(true);
ctx.register_csv("tripdata", path, options).await?
}
"parquet" => {
ctx.register_parquet("tripdata", path, ParquetReadOptions::default())
.await?
}
other => {
println!("Invalid file format '{}'", other);
process::exit(-1);
}
}
datafusion_sql_benchmarks(&mut ctx, opt.iterations, opt.debug).await
}
async fn datafusion_sql_benchmarks(
ctx: &mut SessionContext,
iterations: usize,
debug: bool,
) -> Result<()> {
let mut queries = HashMap::new();
queries.insert("fare_amt_by_passenger", "SELECT passenger_count, MIN(fare_amount), MAX(fare_amount), SUM(fare_amount) FROM tripdata GROUP BY passenger_count");
for (name, sql) in &queries {
println!("Executing '{}'", name);
for i in 0..iterations {
let start = Instant::now();
execute_sql(ctx, sql, debug).await?;
println!(
"Query '{}' iteration {} took {} ms",
name,
i,
start.elapsed().as_millis()
);
}
}
Ok(())
}
async fn execute_sql(ctx: &SessionContext, sql: &str, debug: bool) -> Result<()> {
let plan = ctx.create_logical_plan(sql)?;
let plan = ctx.optimize(&plan)?;
if debug {
println!("Optimized logical plan:\n{:?}", plan);
}
let physical_plan = ctx.create_physical_plan(&plan).await?;
let task_ctx = ctx.task_ctx();
let result = collect(physical_plan, task_ctx).await?;
if debug {
pretty::print_batches(&result)?;
}
Ok(())
}
fn nyctaxi_schema() -> Schema {
Schema::new(vec![
Field::new("VendorID", DataType::Utf8, true),
Field::new("tpep_pickup_datetime", DataType::Utf8, true),
Field::new("tpep_dropoff_datetime", DataType::Utf8, true),
Field::new("passenger_count", DataType::Int32, true),
Field::new("trip_distance", DataType::Utf8, true),
Field::new("RatecodeID", DataType::Utf8, true),
Field::new("store_and_fwd_flag", DataType::Utf8, true),
Field::new("PULocationID", DataType::Utf8, true),
Field::new("DOLocationID", DataType::Utf8, true),
Field::new("payment_type", DataType::Utf8, true),
Field::new("fare_amount", DataType::Float64, true),
Field::new("extra", DataType::Float64, true),
Field::new("mta_tax", DataType::Float64, true),
Field::new("tip_amount", DataType::Float64, true),
Field::new("tolls_amount", DataType::Float64, true),
Field::new("improvement_surcharge", DataType::Float64, true),
Field::new("total_amount", DataType::Float64, true),
])
}