diff --git a/.travis.yml b/.travis.yml index ba5581751..214284f96 100644 --- a/.travis.yml +++ b/.travis.yml @@ -9,19 +9,13 @@ matrix: fast_finish: true include: # NB: To help with CI delays, each `pull_request` is only tested on Linux, - # with 1.13 for compatibility and stable+rayon_unstable for broad test + # with 1.26 for compatibility and stable+rayon_unstable for broad test # coverage. The bors bot counts as a `push` type, which will run it all. - - rust: 1.13.0 + - rust: 1.26.0 os: linux #if: everything! script: cargo build - before_script: - # lazy_static 1.1 requires Rust 1.21+, so downgrade it. - # (and docopt 1.1 requires lazy_static 1.3) - - cargo generate-lockfile - - cargo update -p docopt --precise 1.0.2 - - cargo update -p lazy_static --precise 1.0.2 - rust: stable os: linux diff --git a/Cargo.toml b/Cargo.toml index e7172ea57..a4dec8c70 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -1,7 +1,7 @@ [package] name = "rayon" # Reminder to update html_rool_url in lib.rs when updating version -version = "1.0.3" +version = "1.1.0" authors = ["Niko Matsakis ", "Josh Stone "] description = "Simple work-stealing parallelism for Rust" @@ -11,7 +11,6 @@ documentation = "https://docs.rs/rayon/" readme = "README.md" keywords = ["parallel", "thread", "concurrency", "join", "performance"] categories = ["concurrency"] -build = "build.rs" exclude = ["/ci/*", "/scripts/*", "/.travis.yml", "/appveyor.yml", "/bors.toml"] [workspace] @@ -19,8 +18,8 @@ members = ["rayon-demo", "rayon-core", "rayon-futures"] exclude = ["ci"] [dependencies] -rayon-core = { version = "1.4.1", path = "rayon-core" } -crossbeam-deque = "0.2.0" +rayon-core = { version = "1.5.0", path = "rayon-core" } +crossbeam-deque = "0.6.3" # This is a public dependency! [dependencies.either] diff --git a/README.md b/README.md index c669e7ad2..cf20e7713 100644 --- a/README.md +++ b/README.md @@ -90,7 +90,7 @@ just add: use rayon::prelude::*; ``` -Rayon currently requires `rustc 1.13.0` or greater. +Rayon currently requires `rustc 1.26.0` or greater. ## Contribution diff --git a/build.rs b/build.rs deleted file mode 100644 index f9f7694ec..000000000 --- a/build.rs +++ /dev/null @@ -1,33 +0,0 @@ -use std::env; -use std::io::Write; -use std::process::{Command, Stdio}; - -fn main() { - if probe("fn main() { 0i128; }") { - println!("cargo:rustc-cfg=has_i128"); - } -} - -/// Test if a code snippet can be compiled -fn probe(code: &str) -> bool { - let rustc = env::var_os("RUSTC").unwrap_or_else(|| "rustc".into()); - let out_dir = env::var_os("OUT_DIR").expect("environment variable OUT_DIR"); - - let mut child = Command::new(rustc) - .arg("--out-dir") - .arg(out_dir) - .arg("--emit=obj") - .arg("-") - .stdin(Stdio::piped()) - .spawn() - .expect("rustc probe"); - - child - .stdin - .as_mut() - .expect("rustc stdin") - .write_all(code.as_bytes()) - .expect("write rustc stdin"); - - child.wait().expect("rustc probe").success() -} diff --git a/examples/cpu_monitor.rs b/examples/cpu_monitor.rs index 294eaa0c7..5e049e692 100644 --- a/examples/cpu_monitor.rs +++ b/examples/cpu_monitor.rs @@ -8,7 +8,7 @@ use docopt::Docopt; use std::io; use std::process; -const USAGE: &'static str = " +const USAGE: &str = " Usage: cpu_monitor [options] cpu_monitor --help @@ -76,7 +76,7 @@ fn tasks_ended(args: &Args) { } fn task_stall_root(args: &Args) { - rayon::join(|| task(args), || wait_for_user()); + rayon::join(|| task(args), wait_for_user); } fn task_stall_scope(args: &Args) { diff --git a/rayon-core/Cargo.toml b/rayon-core/Cargo.toml index d474bcb8c..09cfbafbb 100644 --- a/rayon-core/Cargo.toml +++ b/rayon-core/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "rayon-core" -version = "1.4.1" # reminder to update html_root_url attribute +version = "1.5.0" # reminder to update html_root_url attribute authors = ["Niko Matsakis ", "Josh Stone "] description = "Core APIs for Rayon" @@ -13,24 +13,20 @@ readme = "README.md" keywords = ["parallel", "thread", "concurrency", "join", "performance"] categories = ["concurrency"] +# Some dependencies may not be their latest version, in order to support older rustc. [dependencies] num_cpus = "1.2" -libc = "0.2.16" lazy_static = "1" - -# This is deliberately not the latest version, because we want -# to support older rustc than crossbeam-deque 0.3+ does. -[dependencies.crossbeam-deque] -version = "0.2.0" - -# Also held back for rustc compatibility -[dependencies.crossbeam] -version = "0.3.0" +crossbeam-deque = "0.6.3" +crossbeam-queue = "0.1.2" [dev-dependencies] rand = "0.6" rand_xorshift = "0.1" +[target.'cfg(unix)'.dev-dependencies] +libc = "0.2" + [[test]] name = "stack_overflow_crash" path = "tests/stack_overflow_crash.rs" diff --git a/rayon-core/README.md b/rayon-core/README.md index 8267d954a..a3b0268d4 100644 --- a/rayon-core/README.md +++ b/rayon-core/README.md @@ -8,4 +8,4 @@ Please see [Rayon Docs] for details about using Rayon. [Rayon Docs]: https://docs.rs/rayon/ -Rayon-core currently requires `rustc 1.13.0` or greater. +Rayon-core currently requires `rustc 1.26.0` or greater. diff --git a/rayon-core/src/internal/worker.rs b/rayon-core/src/internal/worker.rs index 59990a7d4..fdf31ac8d 100644 --- a/rayon-core/src/internal/worker.rs +++ b/rayon-core/src/internal/worker.rs @@ -60,15 +60,8 @@ pub fn if_in_worker_thread(if_true: F) -> Option where F: FnOnce(&WorkerThread) -> R, { - let worker_thread = registry::WorkerThread::current(); - if worker_thread.is_null() { - None - } else { - unsafe { - let wt = WorkerThread { - thread: &*worker_thread, - }; - Some(if_true(&wt)) - } + unsafe { + let thread = registry::WorkerThread::current().as_ref()?; + Some(if_true(&WorkerThread { thread })) } } diff --git a/rayon-core/src/job.rs b/rayon-core/src/job.rs index bcda77398..a47f4558d 100644 --- a/rayon-core/src/job.rs +++ b/rayon-core/src/job.rs @@ -1,11 +1,11 @@ -use crossbeam::sync::SegQueue; +use crossbeam_queue::SegQueue; use latch::Latch; use std::any::Any; use std::cell::UnsafeCell; use std::mem; use unwind; -pub enum JobResult { +pub(super) enum JobResult { None, Ok(T), Panic(Box), @@ -16,7 +16,7 @@ pub enum JobResult { /// arranged in a deque, so that thieves can take from the top of the /// deque while the main worker manages the bottom of the deque. This /// deque is managed by the `thread_pool` module. -pub trait Job { +pub(super) trait Job { /// Unsafe: this may be called from a different thread than the one /// which scheduled the job, so the implementer must ensure the /// appropriate traits are met, whether `Send`, `Sync`, or both. @@ -30,7 +30,7 @@ pub trait Job { /// true type is something like `*const StackJob<...>`, but we hide /// it. We also carry the "execute fn" from the `Job` trait. #[derive(Copy, Clone, Debug, PartialEq, Eq)] -pub struct JobRef { +pub(super) struct JobRef { pointer: *const (), execute_fn: unsafe fn(*const ()), } @@ -41,24 +41,21 @@ unsafe impl Sync for JobRef {} impl JobRef { /// Unsafe: caller asserts that `data` will remain valid until the /// job is executed. - pub unsafe fn new(data: *const T) -> JobRef + pub(super) unsafe fn new(data: *const T) -> JobRef where T: Job, { let fn_ptr: unsafe fn(*const T) = ::execute; // erase types: - let fn_ptr: unsafe fn(*const ()) = mem::transmute(fn_ptr); - let pointer = data as *const (); - JobRef { - pointer: pointer, - execute_fn: fn_ptr, + pointer: data as *const (), + execute_fn: mem::transmute(fn_ptr), } } #[inline] - pub unsafe fn execute(&self) { + pub(super) unsafe fn execute(&self) { (self.execute_fn)(self.pointer) } } @@ -67,13 +64,13 @@ impl JobRef { /// executes it need not free any heap data, the cleanup occurs when /// the stack frame is later popped. The function parameter indicates /// `true` if the job was stolen -- executed on a different thread. -pub struct StackJob +pub(super) struct StackJob where L: Latch + Sync, F: FnOnce(bool) -> R + Send, R: Send, { - pub latch: L, + pub(super) latch: L, func: UnsafeCell>, result: UnsafeCell>, } @@ -84,23 +81,23 @@ where F: FnOnce(bool) -> R + Send, R: Send, { - pub fn new(func: F, latch: L) -> StackJob { + pub(super) fn new(func: F, latch: L) -> StackJob { StackJob { - latch: latch, + latch, func: UnsafeCell::new(Some(func)), result: UnsafeCell::new(JobResult::None), } } - pub unsafe fn as_job_ref(&self) -> JobRef { + pub(super) unsafe fn as_job_ref(&self) -> JobRef { JobRef::new(self) } - pub unsafe fn run_inline(self, stolen: bool) -> R { + pub(super) unsafe fn run_inline(self, stolen: bool) -> R { self.func.into_inner().unwrap()(stolen) } - pub unsafe fn into_result(self) -> R { + pub(super) unsafe fn into_result(self) -> R { self.result.into_inner().into_return_value() } } @@ -130,7 +127,7 @@ where /// signal that the job executed. /// /// (Probably `StackJob` should be refactored in a similar fashion.) -pub struct HeapJob +pub(super) struct HeapJob where BODY: FnOnce() + Send, { @@ -141,7 +138,7 @@ impl HeapJob where BODY: FnOnce() + Send, { - pub fn new(func: BODY) -> Self { + pub(super) fn new(func: BODY) -> Self { HeapJob { job: UnsafeCell::new(Some(func)), } @@ -150,7 +147,7 @@ where /// Creates a `JobRef` from this job -- note that this hides all /// lifetimes, so it is up to you to ensure that this JobRef /// doesn't outlive any data that it closes over. - pub unsafe fn as_job_ref(self: Box) -> JobRef { + pub(super) unsafe fn as_job_ref(self: Box) -> JobRef { let this: *const Self = mem::transmute(self); JobRef::new(this) } @@ -172,7 +169,7 @@ impl JobResult { /// its JobResult is populated) into its return value. /// /// NB. This will panic if the job panicked. - pub fn into_return_value(self) -> T { + pub(super) fn into_return_value(self) -> T { match self { JobResult::None => unreachable!(), JobResult::Ok(x) => x, @@ -182,18 +179,18 @@ impl JobResult { } /// Indirect queue to provide FIFO job priority. -pub struct JobFifo { +pub(super) struct JobFifo { inner: SegQueue, } impl JobFifo { - pub fn new() -> Self { + pub(super) fn new() -> Self { JobFifo { inner: SegQueue::new(), } } - pub unsafe fn push(&self, job_ref: JobRef) -> JobRef { + pub(super) unsafe fn push(&self, job_ref: JobRef) -> JobRef { // A little indirection ensures that spawns are always prioritized in FIFO order. The // jobs in a thread's deque may be popped from the back (LIFO) or stolen from the front // (FIFO), but either way they will end up popping from the front of this queue. @@ -205,10 +202,6 @@ impl JobFifo { impl Job for JobFifo { unsafe fn execute(this: *const Self) { // We "execute" a queue by executing its first job, FIFO. - (*this) - .inner - .try_pop() - .expect("job in fifo queue") - .execute() + (*this).inner.pop().expect("job in fifo queue").execute() } } diff --git a/rayon-core/src/join/mod.rs b/rayon-core/src/join/mod.rs index 4498bdea3..b5ec331f6 100644 --- a/rayon-core/src/join/mod.rs +++ b/rayon-core/src/join/mod.rs @@ -171,7 +171,7 @@ where } } - return (result_a, job_b.into_result()); + (result_a, job_b.into_result()) }) } diff --git a/rayon-core/src/latch.rs b/rayon-core/src/latch.rs index e16519e4d..ef93fec03 100644 --- a/rayon-core/src/latch.rs +++ b/rayon-core/src/latch.rs @@ -30,12 +30,12 @@ use sleep::Sleep; /// - Once `set()` occurs, the next `probe()` *will* observe it. This /// typically requires a seq-cst ordering. See [the "tickle-then-get-sleepy" scenario in the sleep /// README](/src/sleep/README.md#tickle-then-get-sleepy) for details. -pub trait Latch: LatchProbe { +pub(super) trait Latch: LatchProbe { /// Set the latch, signalling others. fn set(&self); } -pub trait LatchProbe { +pub(super) trait LatchProbe { /// Test if the latch is set. fn probe(&self) -> bool; } @@ -43,13 +43,13 @@ pub trait LatchProbe { /// Spin latches are the simplest, most efficient kind, but they do /// not support a `wait()` operation. They just have a boolean flag /// that becomes true when `set()` is called. -pub struct SpinLatch { +pub(super) struct SpinLatch { b: AtomicBool, } impl SpinLatch { #[inline] - pub fn new() -> SpinLatch { + pub(super) fn new() -> SpinLatch { SpinLatch { b: AtomicBool::new(false), } @@ -72,14 +72,14 @@ impl Latch for SpinLatch { /// A Latch starts as false and eventually becomes true. You can block /// until it becomes true. -pub struct LockLatch { +pub(super) struct LockLatch { m: Mutex, v: Condvar, } impl LockLatch { #[inline] - pub fn new() -> LockLatch { + pub(super) fn new() -> LockLatch { LockLatch { m: Mutex::new(false), v: Condvar::new(), @@ -87,7 +87,7 @@ impl LockLatch { } /// Block until latch is set. - pub fn wait(&self) { + pub(super) fn wait(&self) { let mut guard = self.m.lock().unwrap(); while !*guard { guard = self.v.wait(guard).unwrap(); @@ -119,20 +119,20 @@ impl Latch for LockLatch { /// decrements the counter. The latch is only "set" (in the sense that /// `probe()` returns true) once the counter reaches zero. #[derive(Debug)] -pub struct CountLatch { +pub(super) struct CountLatch { counter: AtomicUsize, } impl CountLatch { #[inline] - pub fn new() -> CountLatch { + pub(super) fn new() -> CountLatch { CountLatch { counter: AtomicUsize::new(1), } } #[inline] - pub fn increment(&self) { + pub(super) fn increment(&self) { debug_assert!(!self.probe()); self.counter.fetch_add(1, Ordering::Relaxed); } @@ -157,17 +157,17 @@ impl Latch for CountLatch { /// A tickling latch wraps another latch type, and will also awaken a thread /// pool when it is set. This is useful for jobs injected between thread pools, /// so the source pool can continue processing its own work while waiting. -pub struct TickleLatch<'a, L: Latch> { +pub(super) struct TickleLatch<'a, L: Latch> { inner: L, sleep: &'a Sleep, } impl<'a, L: Latch> TickleLatch<'a, L> { #[inline] - pub fn new(latch: L, sleep: &'a Sleep) -> Self { + pub(super) fn new(latch: L, sleep: &'a Sleep) -> Self { TickleLatch { inner: latch, - sleep: sleep, + sleep, } } } diff --git a/rayon-core/src/lib.rs b/rayon-core/src/lib.rs index c822eebaa..f1b7fda6c 100644 --- a/rayon-core/src/lib.rs +++ b/rayon-core/src/lib.rs @@ -19,9 +19,10 @@ //! conflicting requirements will need to be resolved before the build will //! succeed. -#![doc(html_root_url = "https://docs.rs/rayon-core/1.4")] +#![doc(html_root_url = "https://docs.rs/rayon-core/1.5")] #![deny(missing_debug_implementations)] #![deny(missing_docs)] +#![deny(unreachable_pub)] use std::any::Any; use std::env; @@ -31,11 +32,11 @@ use std::io; use std::marker::PhantomData; use std::str::FromStr; -extern crate crossbeam; extern crate crossbeam_deque; +extern crate crossbeam_queue; +#[cfg(any(debug_assertions, rayon_unstable))] #[macro_use] extern crate lazy_static; -extern crate libc; extern crate num_cpus; #[cfg(test)] @@ -181,7 +182,7 @@ impl ThreadPoolBuilder { /// Create a new `ThreadPool` initialized using this configuration. pub fn build(self) -> Result { - thread_pool::build(self) + ThreadPool::build(self) } /// Initializes the global thread pool. This initialization is @@ -202,7 +203,7 @@ impl ThreadPoolBuilder { /// will return an error. An `Ok` result indicates that this /// is the first initialization of the thread pool. pub fn build_global(self) -> Result<(), ThreadPoolBuildError> { - let registry = try!(registry::init_global_registry(self)); + let registry = registry::init_global_registry(self)?; registry.wait_until_primed(); Ok(()) } @@ -235,7 +236,8 @@ impl ThreadPoolBuilder { /// Get the thread name for the thread with the given index. fn get_thread_name(&mut self, index: usize) -> Option { - self.get_thread_name.as_mut().map(|c| c(index)) + let f = self.get_thread_name.as_mut()?; + Some(f(index)) } /// Set a closure which takes a thread index and returns @@ -405,7 +407,7 @@ impl Configuration { /// Deprecated in favor of `ThreadPoolBuilder::build`. pub fn build(self) -> Result> { - self.builder.build().map_err(|e| e.into()) + self.builder.build().map_err(Box::from) } /// Deprecated in favor of `ThreadPoolBuilder::thread_name`. @@ -470,7 +472,7 @@ impl Configuration { impl ThreadPoolBuildError { fn new(kind: ErrorKind) -> ThreadPoolBuildError { - ThreadPoolBuildError { kind: kind } + ThreadPoolBuildError { kind } } } @@ -498,7 +500,7 @@ impl fmt::Display for ThreadPoolBuildError { #[deprecated(note = "use `ThreadPoolBuilder::build_global`")] #[allow(deprecated)] pub fn initialize(config: Configuration) -> Result<(), Box> { - config.into_builder().build_global().map_err(|e| e.into()) + config.into_builder().build_global().map_err(Box::from) } impl fmt::Debug for ThreadPoolBuilder { @@ -567,7 +569,7 @@ impl FnContext { #[inline] fn new(migrated: bool) -> Self { FnContext { - migrated: migrated, + migrated, _marker: PhantomData, } } diff --git a/rayon-core/src/log.rs b/rayon-core/src/log.rs index d3a9ebb39..053d37dad 100644 --- a/rayon-core/src/log.rs +++ b/rayon-core/src/log.rs @@ -8,10 +8,12 @@ //! replacement of the now deprecated `RAYON_RS_LOG` environment //! variable, which is still supported for backwards compatibility. +#[cfg(debug_assertions)] use std::env; -#[derive(Debug)] -pub enum Event { +#[cfg_attr(debug_assertions, derive(Debug))] +#[cfg_attr(not(debug_assertions), allow(dead_code))] +pub(super) enum Event { Tickle { worker: usize, old_state: usize, @@ -87,19 +89,28 @@ pub enum Event { }, } -pub const DUMP_LOGS: bool = cfg!(debug_assertions); - +#[cfg(debug_assertions)] lazy_static! { - pub static ref LOG_ENV: bool = + pub(super) static ref LOG_ENV: bool = env::var("RAYON_LOG").is_ok() || env::var("RAYON_RS_LOG").is_ok(); } +#[cfg(debug_assertions)] +macro_rules! log { + ($event:expr) => { + if *$crate::log::LOG_ENV { + eprintln!("{:?}", $event); + } + }; +} + +#[cfg(not(debug_assertions))] macro_rules! log { ($event:expr) => { - if ::log::DUMP_LOGS { - if *::log::LOG_ENV { - println!("{:?}", $event); - } + if false { + // Expand `$event` so it still appears used, but without + // any of the formatting code to be optimized away. + $event; } }; } diff --git a/rayon-core/src/registry.rs b/rayon-core/src/registry.rs index 1483761ec..0691e3395 100644 --- a/rayon-core/src/registry.rs +++ b/rayon-core/src/registry.rs @@ -1,5 +1,5 @@ -use crossbeam::sync::SegQueue; -use crossbeam_deque::{Deque, Steal, Stealer}; +use crossbeam_deque::{self as deque, Pop, Steal, Stealer, Worker}; +use crossbeam_queue::SegQueue; #[cfg(rayon_unstable)] use internal::task::Task; #[cfg(rayon_unstable)] @@ -13,6 +13,7 @@ use std::cell::Cell; use std::collections::hash_map::DefaultHasher; use std::hash::Hasher; use std::mem; +use std::ptr; #[allow(deprecated)] use std::sync::atomic::ATOMIC_USIZE_INIT; use std::sync::atomic::{AtomicUsize, Ordering}; @@ -23,7 +24,7 @@ use unwind; use util::leak; use {ErrorKind, ExitHandler, PanicHandler, StartHandler, ThreadPoolBuildError, ThreadPoolBuilder}; -pub struct Registry { +pub(super) struct Registry { thread_infos: Vec, sleep: Sleep, injected_jobs: SegQueue, @@ -63,7 +64,7 @@ fn global_registry() -> &'static Arc { /// Starts the worker threads (if that has not already happened) with /// the given builder. -pub fn init_global_registry( +pub(super) fn init_global_registry( builder: ThreadPoolBuilder, ) -> Result<&'static Registry, ThreadPoolBuildError> { let mut called = false; @@ -73,7 +74,8 @@ pub fn init_global_registry( called = true; }); if called { - init_result.map(|()| &**global_registry()) + init_result?; + Ok(&**global_registry()) } else { Err(ThreadPoolBuildError::new( ErrorKind::GlobalPoolAlreadyInitialized, @@ -86,7 +88,9 @@ pub fn init_global_registry( /// function. Declared `unsafe` because it writes to `THE_REGISTRY` in /// an unsynchronized fashion. unsafe fn init_registry(builder: ThreadPoolBuilder) -> Result<(), ThreadPoolBuildError> { - Registry::new(builder).map(|registry| THE_REGISTRY = Some(leak(registry))) + let registry = Registry::new(builder)?; + THE_REGISTRY = Some(leak(registry)); + Ok(()) } struct Terminator<'a>(&'a Arc); @@ -98,15 +102,22 @@ impl<'a> Drop for Terminator<'a> { } impl Registry { - pub fn new(mut builder: ThreadPoolBuilder) -> Result, ThreadPoolBuildError> { + pub(super) fn new(mut builder: ThreadPoolBuilder) -> Result, ThreadPoolBuildError> { let n_threads = builder.get_num_threads(); let breadth_first = builder.get_breadth_first(); - let workers: Vec<_> = (0..n_threads).map(|_| Deque::new()).collect(); - let stealers: Vec<_> = workers.iter().map(|d| d.stealer()).collect(); + let (workers, stealers): (Vec<_>, Vec<_>) = (0..n_threads) + .map(|_| { + if breadth_first { + deque::fifo() + } else { + deque::lifo() + } + }) + .unzip(); let registry = Arc::new(Registry { - thread_infos: stealers.into_iter().map(|s| ThreadInfo::new(s)).collect(), + thread_infos: stealers.into_iter().map(ThreadInfo::new).collect(), sleep: Sleep::new(), injected_jobs: SegQueue::new(), terminate_latch: CountLatch::new(), @@ -127,9 +138,7 @@ impl Registry { if let Some(stack_size) = builder.get_stack_size() { b = b.stack_size(stack_size); } - if let Err(e) = - b.spawn(move || unsafe { main_loop(worker, registry, index, breadth_first) }) - { + if let Err(e) = b.spawn(move || unsafe { main_loop(worker, registry, index) }) { return Err(ThreadPoolBuildError::new(ErrorKind::IOError(e))); } } @@ -141,11 +150,11 @@ impl Registry { } #[cfg(rayon_unstable)] - pub fn global() -> Arc { + pub(super) fn global() -> Arc { global_registry().clone() } - pub fn current() -> Arc { + pub(super) fn current() -> Arc { unsafe { let worker_thread = WorkerThread::current(); if worker_thread.is_null() { @@ -159,7 +168,7 @@ impl Registry { /// Returns the number of threads in the current registry. This /// is better than `Registry::current().num_threads()` because it /// avoids incrementing the `Arc`. - pub fn current_num_threads() -> usize { + pub(super) fn current_num_threads() -> usize { unsafe { let worker_thread = WorkerThread::current(); if worker_thread.is_null() { @@ -171,19 +180,19 @@ impl Registry { } /// Returns the current `WorkerThread` if it's part of this `Registry`. - pub fn current_thread(&self) -> Option<&WorkerThread> { + pub(super) fn current_thread(&self) -> Option<&WorkerThread> { unsafe { - if let Some(worker) = WorkerThread::current().as_ref() { - if worker.registry().id() == self.id() { - return Some(worker); - } + let worker = WorkerThread::current().as_ref()?; + if worker.registry().id() == self.id() { + Some(worker) + } else { + None } - None } } /// Returns an opaque identifier for this registry. - pub fn id(&self) -> RegistryId { + pub(super) fn id(&self) -> RegistryId { // We can rely on `self` not to change since we only ever create // registries that are boxed up in an `Arc` (see `new()` above). RegistryId { @@ -191,11 +200,11 @@ impl Registry { } } - pub fn num_threads(&self) -> usize { + pub(super) fn num_threads(&self) -> usize { self.thread_infos.len() } - pub fn handle_panic(&self, err: Box) { + pub(super) fn handle_panic(&self, err: Box) { match self.panic_handler { Some(ref handler) => { // If the customizable panic handler itself panics, @@ -215,7 +224,7 @@ impl Registry { /// meant to be used for benchmarking purposes, primarily, so that /// you can get more consistent numbers by having everything /// "ready to go". - pub fn wait_until_primed(&self) { + pub(super) fn wait_until_primed(&self) { for info in &self.thread_infos { info.primed.wait(); } @@ -224,7 +233,7 @@ impl Registry { /// Waits for the worker threads to stop. This is used for testing /// -- so we can check that termination actually works. #[cfg(test)] - pub fn wait_until_stopped(&self) { + pub(super) fn wait_until_stopped(&self) { for info in &self.thread_infos { info.stopped.wait(); } @@ -239,7 +248,7 @@ impl Registry { /// Push a job into the given `registry`. If we are running on a /// worker thread for the registry, this will push onto the /// deque. Else, it will inject from the outside (which is slower). - pub fn inject_or_push(&self, job_ref: JobRef) { + pub(super) fn inject_or_push(&self, job_ref: JobRef) { let worker_thread = WorkerThread::current(); unsafe { if !worker_thread.is_null() && (*worker_thread).registry().id() == self.id() { @@ -253,7 +262,7 @@ impl Registry { /// Unsafe: the caller must guarantee that `task` will stay valid /// until it executes. #[cfg(rayon_unstable)] - pub unsafe fn submit_task(&self, task: Arc) + pub(super) unsafe fn submit_task(&self, task: Arc) where T: Task, { @@ -277,7 +286,7 @@ impl Registry { unsafe { mem::transmute(arc) } } - pub fn into_task(this: Arc>) -> Arc { + fn into_task(this: Arc>) -> Arc { // Same logic as `new()` unsafe { mem::transmute(this) } } @@ -300,7 +309,7 @@ impl Registry { /// Push a job into the "external jobs" queue; it will be taken by /// whatever worker has nothing to do. Use this is you know that /// you are not on a worker of this registry. - pub fn inject(&self, injected_jobs: &[JobRef]) { + pub(super) fn inject(&self, injected_jobs: &[JobRef]) { log!(InjectJobs { count: injected_jobs.len() }); @@ -322,7 +331,7 @@ impl Registry { } fn pop_injected_job(&self, worker_index: usize) -> Option { - let job = self.injected_jobs.try_pop(); + let job = self.injected_jobs.pop().ok(); if job.is_some() { log!(UninjectedWork { worker: worker_index @@ -336,7 +345,7 @@ impl Registry { /// completes and return its return value. If `op` panics, that panic will /// be propagated as well. The second argument indicates `true` if injection /// was performed, `false` if executed directly. - pub fn in_worker(&self, op: OP) -> R + pub(super) fn in_worker(&self, op: OP) -> R where OP: FnOnce(&WorkerThread, bool) -> R + Send, R: Send, @@ -420,21 +429,21 @@ impl Registry { /// of any blocking scope. In that case, the job itself holds a /// terminate count and is responsible for invoking `terminate()` /// when finished. - pub fn increment_terminate_count(&self) { + pub(super) fn increment_terminate_count(&self) { self.terminate_latch.increment(); } /// Signals that the thread-pool which owns this registry has been /// dropped. The worker threads will gradually terminate, once any /// extant work is completed. - pub fn terminate(&self) { + pub(super) fn terminate(&self) { self.terminate_latch.set(); self.sleep.tickle(usize::MAX); } } #[derive(Copy, Clone, Debug, PartialEq, Eq, PartialOrd, Ord)] -pub struct RegistryId { +pub(super) struct RegistryId { addr: usize, } @@ -457,7 +466,7 @@ impl ThreadInfo { ThreadInfo { primed: LockLatch::new(), stopped: LockLatch::new(), - stealer: stealer, + stealer, } } } @@ -465,18 +474,15 @@ impl ThreadInfo { /// //////////////////////////////////////////////////////////////////////// /// WorkerThread identifiers -pub struct WorkerThread { +pub(super) struct WorkerThread { /// the "worker" half of our local deque - worker: Deque, + worker: Worker, /// local queue used for `spawn_fifo` indirection fifo: JobFifo, index: usize, - /// are these workers configured to steal breadth-first or not? - breadth_first: bool, - /// A weak random number generator. rng: XorShift64Star, @@ -489,8 +495,7 @@ pub struct WorkerThread { // worker is fully unwound. Using an unsafe pointer avoids the need // for a RefCell etc. thread_local! { - static WORKER_THREAD_STATE: Cell<*const WorkerThread> = - Cell::new(0 as *const WorkerThread) + static WORKER_THREAD_STATE: Cell<*const WorkerThread> = Cell::new(ptr::null()); } impl WorkerThread { @@ -498,8 +503,8 @@ impl WorkerThread { /// NULL if this is not a worker thread. This pointer is valid /// anywhere on the current thread. #[inline] - pub fn current() -> *const WorkerThread { - WORKER_THREAD_STATE.with(|t| t.get()) + pub(super) fn current() -> *const WorkerThread { + WORKER_THREAD_STATE.with(Cell::get) } /// Sets `self` as the worker thread index for the current thread. @@ -512,30 +517,30 @@ impl WorkerThread { } /// Returns the registry that owns this worker thread. - pub fn registry(&self) -> &Arc { + pub(super) fn registry(&self) -> &Arc { &self.registry } /// Our index amongst the worker threads (ranges from `0..self.num_threads()`). #[inline] - pub fn index(&self) -> usize { + pub(super) fn index(&self) -> usize { self.index } #[inline] - pub unsafe fn push(&self, job: JobRef) { + pub(super) unsafe fn push(&self, job: JobRef) { self.worker.push(job); self.registry.sleep.tickle(self.index); } #[inline] - pub unsafe fn push_fifo(&self, job: JobRef) { + pub(super) unsafe fn push_fifo(&self, job: JobRef) { self.push(self.fifo.push(job)); } #[inline] - pub fn local_deque_is_empty(&self) -> bool { - self.worker.len() == 0 + pub(super) fn local_deque_is_empty(&self) -> bool { + self.worker.is_empty() } /// Attempts to obtain a "local" job -- typically this means @@ -543,16 +548,12 @@ impl WorkerThread { /// for breadth-first execution, it would mean dequeuing from the /// bottom. #[inline] - pub unsafe fn take_local_job(&self) -> Option { - if !self.breadth_first { - self.worker.pop() - } else { - loop { - match self.worker.steal() { - Steal::Empty => return None, - Steal::Data(d) => return Some(d), - Steal::Retry => {} - } + pub(super) unsafe fn take_local_job(&self) -> Option { + loop { + match self.worker.pop() { + Pop::Empty => return None, + Pop::Data(d) => return Some(d), + Pop::Retry => {} } } } @@ -560,7 +561,7 @@ impl WorkerThread { /// Wait until the latch is set. Try to keep busy by popping and /// stealing tasks as necessary. #[inline] - pub unsafe fn wait_until(&self, latch: &L) { + pub(super) unsafe fn wait_until(&self, latch: &L) { log!(WaitUntil { worker: self.index }); if !latch.probe() { self.wait_until_cold(latch); @@ -604,7 +605,7 @@ impl WorkerThread { mem::forget(abort_guard); // successful execution, do not abort } - pub unsafe fn execute(&self, job: JobRef) { + pub(super) unsafe fn execute(&self, job: JobRef) { job.execute(); // Subtle: executing this job will have `set()` some of its @@ -620,7 +621,7 @@ impl WorkerThread { /// local work to do. unsafe fn steal(&self) -> Option { // we only steal when we don't have any work to do locally - debug_assert!(self.worker.pop().is_none()); + debug_assert!(self.local_deque_is_empty()); // otherwise, try to steal let num_threads = self.registry.thread_infos.len(); @@ -654,17 +655,11 @@ impl WorkerThread { /// //////////////////////////////////////////////////////////////////////// -unsafe fn main_loop( - worker: Deque, - registry: Arc, - index: usize, - breadth_first: bool, -) { +unsafe fn main_loop(worker: Worker, registry: Arc, index: usize) { let worker_thread = WorkerThread { - worker: worker, + worker, fifo: JobFifo::new(), - breadth_first: breadth_first, - index: index, + index, rng: XorShift64Star::new(), registry: registry.clone(), }; @@ -718,7 +713,7 @@ unsafe fn main_loop( /// `op` completes and return its return value. If `op` panics, that /// panic will be propagated as well. The second argument indicates /// `true` if injection was performed, `false` if executed directly. -pub fn in_worker(op: OP) -> R +pub(super) fn in_worker(op: OP) -> R where OP: FnOnce(&WorkerThread, bool) -> R + Send, R: Send, diff --git a/rayon-core/src/scope/mod.rs b/rayon-core/src/scope/mod.rs index 7cf25177d..a797beae0 100644 --- a/rayon-core/src/scope/mod.rs +++ b/rayon-core/src/scope/mod.rs @@ -624,7 +624,7 @@ impl<'scope> ScopeBase<'scope> { } impl<'scope> fmt::Debug for Scope<'scope> { - fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { fmt.debug_struct("Scope") .field("pool_id", &self.base.registry.id()) .field("owner_thread_index", &self.base.owner_thread_index) diff --git a/rayon-core/src/scope/test.rs b/rayon-core/src/scope/test.rs index 1de609570..37f5e29a0 100644 --- a/rayon-core/src/scope/test.rs +++ b/rayon-core/src/scope/test.rs @@ -75,14 +75,14 @@ struct Tree { } impl Tree { - pub fn iter<'s>(&'s self) -> vec::IntoIter<&'s T> { + fn iter<'s>(&'s self) -> vec::IntoIter<&'s T> { once(&self.value) - .chain(self.children.iter().flat_map(|c| c.iter())) + .chain(self.children.iter().flat_map(Tree::iter)) .collect::>() // seems like it shouldn't be needed... but prevents overflow .into_iter() } - pub fn update(&mut self, op: OP) + fn update(&mut self, op: OP) where OP: Fn(&mut T) + Sync, T: Send, @@ -127,7 +127,7 @@ fn random_tree1(depth: usize, rng: &mut XorShiftRng) -> Tree { Tree { value: rng.gen_range(0, 1_000_000), - children: children, + children, } } diff --git a/rayon-core/src/sleep/mod.rs b/rayon-core/src/sleep/mod.rs index 7582f98f7..96f4c37ff 100644 --- a/rayon-core/src/sleep/mod.rs +++ b/rayon-core/src/sleep/mod.rs @@ -7,7 +7,7 @@ use std::sync::{Condvar, Mutex}; use std::thread; use std::usize; -pub struct Sleep { +pub(super) struct Sleep { state: AtomicUsize, data: Mutex<()>, tickle: Condvar, @@ -20,7 +20,7 @@ const ROUNDS_UNTIL_SLEEPY: usize = 32; const ROUNDS_UNTIL_ASLEEP: usize = 64; impl Sleep { - pub fn new() -> Sleep { + pub(super) fn new() -> Sleep { Sleep { state: AtomicUsize::new(AWAKE), data: Mutex::new(()), @@ -46,7 +46,7 @@ impl Sleep { } #[inline] - pub fn work_found(&self, worker_index: usize, yields: usize) -> usize { + pub(super) fn work_found(&self, worker_index: usize, yields: usize) -> usize { log!(FoundWork { worker: worker_index, yields: yields, @@ -61,7 +61,7 @@ impl Sleep { } #[inline] - pub fn no_work_found(&self, worker_index: usize, yields: usize) -> usize { + pub(super) fn no_work_found(&self, worker_index: usize, yields: usize) -> usize { log!(DidNotFindWork { worker: worker_index, yields: yields, @@ -93,7 +93,7 @@ impl Sleep { } } - pub fn tickle(&self, worker_index: usize) { + pub(super) fn tickle(&self, worker_index: usize) { // As described in README.md, this load must be SeqCst so as to ensure that: // - if anyone is sleepy or asleep, we *definitely* see that now (and not eventually); // - if anyone after us becomes sleepy or asleep, they see memory events that diff --git a/rayon-core/src/spawn/mod.rs b/rayon-core/src/spawn/mod.rs index c2194e3f7..37b2d1b36 100644 --- a/rayon-core/src/spawn/mod.rs +++ b/rayon-core/src/spawn/mod.rs @@ -69,9 +69,7 @@ where /// Spawn an asynchronous job in `registry.` /// /// Unsafe because `registry` must not yet have terminated. -/// -/// Not a public API, but used elsewhere in Rayon. -pub unsafe fn spawn_in(func: F, registry: &Arc) +pub(super) unsafe fn spawn_in(func: F, registry: &Arc) where F: FnOnce() + Send + 'static, { @@ -146,9 +144,7 @@ where /// Spawn an asynchronous FIFO job in `registry.` /// /// Unsafe because `registry` must not yet have terminated. -/// -/// Not a public API, but used elsewhere in Rayon. -pub unsafe fn spawn_fifo_in(func: F, registry: &Arc) +pub(super) unsafe fn spawn_fifo_in(func: F, registry: &Arc) where F: FnOnce() + Send + 'static, { diff --git a/rayon-core/src/thread_pool/internal.rs b/rayon-core/src/thread_pool/internal.rs index 58d41cc8b..29e439026 100644 --- a/rayon-core/src/thread_pool/internal.rs +++ b/rayon-core/src/thread_pool/internal.rs @@ -31,7 +31,7 @@ impl ThreadPoolScopeHandle { /// Caller asserts that the registry has not yet terminated. unsafe fn new(registry: Arc) -> Self { registry.increment_terminate_count(); - ThreadPoolScopeHandle { registry: registry } + ThreadPoolScopeHandle { registry } } } diff --git a/rayon-core/src/thread_pool/mod.rs b/rayon-core/src/thread_pool/mod.rs index 0e90c84aa..3834a1dbc 100644 --- a/rayon-core/src/thread_pool/mod.rs +++ b/rayon-core/src/thread_pool/mod.rs @@ -53,17 +53,17 @@ pub struct ThreadPool { registry: Arc, } -pub fn build(builder: ThreadPoolBuilder) -> Result { - let registry = try!(Registry::new(builder)); - Ok(ThreadPool { registry: registry }) -} - impl ThreadPool { #[deprecated(note = "Use `ThreadPoolBuilder::build`")] #[allow(deprecated)] /// Deprecated in favor of `ThreadPoolBuilder::build`. pub fn new(configuration: Configuration) -> Result> { - build(configuration.into_builder()).map_err(|e| e.into()) + Self::build(configuration.into_builder()).map_err(Box::from) + } + + pub(super) fn build(builder: ThreadPoolBuilder) -> Result { + let registry = Registry::new(builder)?; + Ok(ThreadPool { registry }) } /// Returns a handle to the global thread pool. This is the pool @@ -165,16 +165,8 @@ impl ThreadPool { /// [snt]: struct.ThreadPoolBuilder.html#method.num_threads #[inline] pub fn current_thread_index(&self) -> Option { - unsafe { - let curr = WorkerThread::current(); - if curr.is_null() { - None - } else if (*curr).registry().id() != self.registry.id() { - None - } else { - Some((*curr).index()) - } - } + let curr = self.registry.current_thread()?; + Some(curr.index()) } /// Returns true if the current worker thread currently has "local @@ -200,16 +192,8 @@ impl ThreadPool { /// [deque]: https://en.wikipedia.org/wiki/Double-ended_queue #[inline] pub fn current_thread_has_pending_tasks(&self) -> Option { - unsafe { - let curr = WorkerThread::current(); - if curr.is_null() { - None - } else if (*curr).registry().id() != self.registry.id() { - None - } else { - Some(!(*curr).local_deque_is_empty()) - } - } + let curr = self.registry.current_thread()?; + Some(!curr.local_deque_is_empty()) } /// Execute `oper_a` and `oper_b` in the thread-pool and return @@ -328,12 +312,8 @@ impl fmt::Debug for ThreadPool { #[inline] pub fn current_thread_index() -> Option { unsafe { - let curr = WorkerThread::current(); - if curr.is_null() { - None - } else { - Some((*curr).index()) - } + let curr = WorkerThread::current().as_ref()?; + Some(curr.index()) } } @@ -346,11 +326,7 @@ pub fn current_thread_index() -> Option { #[inline] pub fn current_thread_has_pending_tasks() -> Option { unsafe { - let curr = WorkerThread::current(); - if curr.is_null() { - None - } else { - Some(!(*curr).local_deque_is_empty()) - } + let curr = WorkerThread::current().as_ref()?; + Some(!curr.local_deque_is_empty()) } } diff --git a/rayon-core/src/thread_pool/test.rs b/rayon-core/src/thread_pool/test.rs index 87f31def9..c1ddca982 100644 --- a/rayon-core/src/thread_pool/test.rs +++ b/rayon-core/src/thread_pool/test.rs @@ -68,14 +68,11 @@ fn sleeper_stop() { } /// Create a start/exit handler that increments an atomic counter. -fn count_handler() -> (Arc, Box<::StartHandler>) { +fn count_handler() -> (Arc, impl Fn(usize)) { let count = Arc::new(AtomicUsize::new(0)); - ( - count.clone(), - Box::new(move |_| { - count.fetch_add(1, Ordering::SeqCst); - }), - ) + (count.clone(), move |_| { + count.fetch_add(1, Ordering::SeqCst); + }) } /// Wait until a counter is no longer shared, then return its value. @@ -109,8 +106,8 @@ fn failed_thread_stack() { let builder = ThreadPoolBuilder::new() .num_threads(10) .stack_size(stack_size) - .start_handler(move |i| start_handler(i)) - .exit_handler(move |i| exit_handler(i)); + .start_handler(start_handler) + .exit_handler(exit_handler); let pool = builder.build(); assert!(pool.is_err(), "thread stack should have failed!"); @@ -128,8 +125,8 @@ fn panic_thread_name() { let (exit_count, exit_handler) = count_handler(); let builder = ThreadPoolBuilder::new() .num_threads(10) - .start_handler(move |i| start_handler(i)) - .exit_handler(move |i| exit_handler(i)) + .start_handler(start_handler) + .exit_handler(exit_handler) .thread_name(|i| { if i >= 5 { panic!(); diff --git a/rayon-core/src/unwind.rs b/rayon-core/src/unwind.rs index 5aeece990..fceed410b 100644 --- a/rayon-core/src/unwind.rs +++ b/rayon-core/src/unwind.rs @@ -3,8 +3,6 @@ //! accidental panics in the rayon code itself. use std::any::Any; -use std::io::prelude::*; -use std::io::stderr; use std::panic::{self, AssertUnwindSafe}; use std::thread; @@ -12,35 +10,22 @@ use std::thread; /// `Err` result. The assumption is that any panic will be propagated /// later with `resume_unwinding`, and hence `f` can be treated as /// exception safe. -pub fn halt_unwinding(func: F) -> thread::Result +pub(super) fn halt_unwinding(func: F) -> thread::Result where F: FnOnce() -> R, { panic::catch_unwind(AssertUnwindSafe(func)) } -pub fn resume_unwinding(payload: Box) -> ! { +pub(super) fn resume_unwinding(payload: Box) -> ! { panic::resume_unwind(payload) } -pub struct AbortIfPanic; - -fn aborting() { - let _ = writeln!(&mut stderr(), "Rayon: detected unexpected panic; aborting"); -} +pub(super) struct AbortIfPanic; impl Drop for AbortIfPanic { - #[cfg(all(target_arch = "wasm32", not(target_os = "emscripten")))] - fn drop(&mut self) { - aborting(); - ::std::process::abort(); // stable in rust 1.17 - } - - #[cfg(not(all(target_arch = "wasm32", not(target_os = "emscripten"))))] fn drop(&mut self) { - aborting(); - unsafe { - ::libc::abort(); // used for compat before 1.17 - } + eprintln!("Rayon: detected unexpected panic; aborting"); + ::std::process::abort(); } } diff --git a/rayon-core/src/util.rs b/rayon-core/src/util.rs index c61cdb200..27d375a4a 100644 --- a/rayon-core/src/util.rs +++ b/rayon-core/src/util.rs @@ -1,6 +1,6 @@ use std::mem; -pub fn leak(v: T) -> &'static T { +pub(super) fn leak(v: T) -> &'static T { unsafe { let b = Box::new(v); let p: *const T = &*b; diff --git a/rayon-core/tests/stack_overflow_crash.rs b/rayon-core/tests/stack_overflow_crash.rs index 475303e4a..954823bdd 100644 --- a/rayon-core/tests/stack_overflow_crash.rs +++ b/rayon-core/tests/stack_overflow_crash.rs @@ -1,3 +1,5 @@ +#[cfg(unix)] +extern crate libc; extern crate rayon_core; use rayon_core::ThreadPoolBuilder; @@ -15,6 +17,19 @@ fn force_stack_overflow(depth: u32) { } } +#[cfg(unix)] +fn disable_core() { + unsafe { + libc::setrlimit( + libc::RLIMIT_CORE, + &libc::rlimit { + rlim_cur: 0, + rlim_max: 0, + }, + ); + } +} + #[cfg(unix)] fn overflow_code() -> Option { None @@ -63,6 +78,8 @@ fn main() { .build() .unwrap(); pool.install(|| { + #[cfg(unix)] + disable_core(); force_stack_overflow(32); }); } diff --git a/rayon-demo/Cargo.toml b/rayon-demo/Cargo.toml index 609f917c9..bea412f57 100644 --- a/rayon-demo/Cargo.toml +++ b/rayon-demo/Cargo.toml @@ -9,7 +9,7 @@ rayon = { path = "../" } cgmath = "0.17" docopt = "1" fixedbitset = "0.1.5" -glium = "0.23" +glium = "0.24" lazy_static = "1" odds = "0.3" rand = "0.6" diff --git a/rayon-demo/src/factorial/mod.rs b/rayon-demo/src/factorial/mod.rs index f64ba62fe..3ec42ca75 100644 --- a/rayon-demo/src/factorial/mod.rs +++ b/rayon-demo/src/factorial/mod.rs @@ -10,7 +10,7 @@ const N: u32 = 9999; /// Compute the Factorial using a plain iterator. fn factorial(n: u32) -> BigUint { - (1..n + 1).map(BigUint::from).fold(BigUint::one(), Mul::mul) + (1..=n).map(BigUint::from).fold(BigUint::one(), Mul::mul) } #[bench] diff --git a/rayon-demo/src/fibonacci/mod.rs b/rayon-demo/src/fibonacci/mod.rs index 0293b82b1..89e2e8553 100644 --- a/rayon-demo/src/fibonacci/mod.rs +++ b/rayon-demo/src/fibonacci/mod.rs @@ -17,7 +17,7 @@ extern crate rayon; extern crate test; const N: u32 = 32; -const FN: u32 = 2178309; +const FN: u32 = 2_178_309; fn fib_iterative(n: u32) -> u32 { let mut a = 0; diff --git a/rayon-demo/src/life/mod.rs b/rayon-demo/src/life/mod.rs index 0044e54e6..b63cc2126 100644 --- a/rayon-demo/src/life/mod.rs +++ b/rayon-demo/src/life/mod.rs @@ -1,4 +1,4 @@ -const USAGE: &'static str = " +const USAGE: &str = " Usage: life bench [--size N] [--gens N] life play [--size N] [--gens N] [--fps N] life --help @@ -68,8 +68,8 @@ impl Board { board: new_board, born: Arc::new(born), survive: Arc::new(survive), - rows: rows, - cols: cols, + rows, + cols, } } diff --git a/rayon-demo/src/main.rs b/rayon-demo/src/main.rs index 609da0d43..69dcf45c0 100644 --- a/rayon-demo/src/main.rs +++ b/rayon-demo/src/main.rs @@ -60,7 +60,7 @@ extern crate winapi; // life #[cfg(test)] extern crate test; -const USAGE: &'static str = " +const USAGE: &str = " Usage: rayon-demo bench rayon-demo [ options ] rayon-demo --help diff --git a/rayon-demo/src/map_collect.rs b/rayon-demo/src/map_collect.rs index c738fe6cd..af071cb85 100644 --- a/rayon-demo/src/map_collect.rs +++ b/rayon-demo/src/map_collect.rs @@ -41,13 +41,10 @@ mod util { PI: ParallelIterator + Send, { let mutex = Mutex::new(HashMap::new()); - pi.fold( - || Vec::new(), - |mut vec, elem| { - vec.push(elem); - vec - }, - ) + pi.fold(Vec::new, |mut vec, elem| { + vec.push(elem); + vec + }) .for_each(|vec| { let mut guard = mutex.lock().unwrap(); guard.extend(vec); @@ -74,13 +71,10 @@ mod util { PI: ParallelIterator + Send, { let list: LinkedList> = pi - .fold( - || Vec::new(), - |mut vec, elem| { - vec.push(elem); - vec - }, - ) + .fold(Vec::new, |mut vec, elem| { + vec.push(elem); + vec + }) .collect(); list.into_iter().fold(HashMap::new(), |mut map, vec| { map.extend(vec); @@ -96,13 +90,10 @@ mod util { PI: ParallelIterator + Send, { let list: LinkedList> = pi - .fold( - || Vec::new(), - |mut vec, elem| { - vec.push(elem); - vec - }, - ) + .fold(Vec::new, |mut vec, elem| { + vec.push(elem); + vec + }) .collect(); let len = list.iter().map(Vec::len).sum(); @@ -121,13 +112,10 @@ mod util { PI: ParallelIterator + Send, { let list: LinkedList> = pi - .fold( - || Vec::new(), - |mut vec, elem| { - vec.push(elem); - vec - }, - ) + .fold(Vec::new, |mut vec, elem| { + vec.push(elem); + vec + }) .map(|vec| { let mut list = LinkedList::new(); list.push_back(vec); @@ -154,21 +142,15 @@ mod util { PI: ParallelIterator + Send, { let vecs: Vec> = pi - .fold( - || Vec::new(), - |mut vec, elem| { - vec.push(elem); - vec - }, - ) + .fold(Vec::new, |mut vec, elem| { + vec.push(elem); + vec + }) .map(|v| vec![v]) - .reduce( - || Vec::new(), - |mut left, mut right| { - left.append(&mut right); - left - }, - ); + .reduce(Vec::new, |mut left, mut right| { + left.append(&mut right); + left + }); let len = vecs.iter().map(Vec::len).sum(); vecs.into_iter() @@ -185,25 +167,19 @@ mod util { V: Send, PI: ParallelIterator + Send, { - pi.fold( - || HashMap::new(), - |mut map, (k, v)| { - map.insert(k, v); - map - }, - ) - .reduce( - || HashMap::new(), - |mut map1, mut map2| { - if map1.len() > map2.len() { - map1.extend(map2); - map1 - } else { - map2.extend(map1); - map2 - } - }, - ) + pi.fold(HashMap::new, |mut map, (k, v)| { + map.insert(k, v); + map + }) + .reduce(HashMap::new, |mut map1, mut map2| { + if map1.len() > map2.len() { + map1.extend(map2); + map1 + } else { + map2.extend(map1); + map2 + } + }) } /// Fold into vecs and then reduce them together as hashmaps. @@ -213,21 +189,15 @@ mod util { V: Send, PI: ParallelIterator + Send, { - pi.fold( - || Vec::new(), - |mut vec, elem| { - vec.push(elem); - vec - }, - ) + pi.fold(Vec::new, |mut vec, elem| { + vec.push(elem); + vec + }) .map(HashMap::from_iter) - .reduce( - || HashMap::new(), - |mut map1, map2| { - map1.extend(map2); - map1 - }, - ) + .reduce(HashMap::new, |mut map1, map2| { + map1.extend(map2); + map1 + }) } } diff --git a/rayon-demo/src/matmul/mod.rs b/rayon-demo/src/matmul/mod.rs index 21bd17e79..954e8877e 100644 --- a/rayon-demo/src/matmul/mod.rs +++ b/rayon-demo/src/matmul/mod.rs @@ -1,4 +1,4 @@ -const USAGE: &'static str = " +const USAGE: &str = " Usage: matmul bench [--size N] matmul --help Parallel matrix multiplication. @@ -54,7 +54,7 @@ struct SplayedBitsCounter { impl SplayedBitsCounter { fn new(max: usize) -> Self { - SplayedBitsCounter { value: 0, max: max } + SplayedBitsCounter { value: 0, max } } } @@ -63,10 +63,10 @@ impl Iterator for SplayedBitsCounter { fn next(&mut self) -> Option { // Return only odd bits. - let prev = self.value & 0x55555555; + let prev = self.value & 0x5555_5555; if prev < self.max { // Set all even bits. - self.value |= 0xaaaaaaaa; + self.value |= 0xaaaa_aaaa; // Add one, carrying through even bits. self.value += 1; Some(prev) @@ -101,9 +101,9 @@ pub fn seq_matmulz(a: &[f32], b: &[f32], dest: &mut [f32]) { // Multiply in morton order // D[i,j] = sum for all k A[i,k] * B[k,j] let n = dest.len(); - for ij in 0..n { - let i = ij & 0xaaaaaaaa; - let j = ij & 0x55555555; + for (ij, d) in dest.iter_mut().enumerate() { + let i = ij & 0xaaaa_aaaa; + let j = ij & 0x5555_5555; let mut sum = 0.0; for k in SplayedBitsCounter::new(n) { // sum += a[i, k] * b[k, j]; @@ -113,14 +113,14 @@ pub fn seq_matmulz(a: &[f32], b: &[f32], dest: &mut [f32]) { a.get_unchecked(i | k) * b.get_unchecked(k << 1 | j) }; } - dest[ij] = sum; + *d = sum; } } const MULT_CHUNK: usize = 1 * 1024; const LINEAR_CHUNK: usize = 64 * 1024; -fn quarter_chunks<'a>(v: &'a [f32]) -> (&'a [f32], &'a [f32], &'a [f32], &'a [f32]) { +fn quarter_chunks(v: &[f32]) -> (&[f32], &[f32], &[f32], &[f32]) { let mid = v.len() / 2; let quarter = mid / 2; let (left, right) = v.split_at(mid); @@ -129,9 +129,7 @@ fn quarter_chunks<'a>(v: &'a [f32]) -> (&'a [f32], &'a [f32], &'a [f32], &'a [f3 (a, b, c, d) } -fn quarter_chunks_mut<'a>( - v: &'a mut [f32], -) -> (&'a mut [f32], &'a mut [f32], &'a mut [f32], &'a mut [f32]) { +fn quarter_chunks_mut(v: &mut [f32]) -> (&mut [f32], &mut [f32], &mut [f32], &mut [f32]) { let mid = v.len() / 2; let quarter = mid / 2; let (left, right) = v.split_at_mut(mid); @@ -390,7 +388,7 @@ fn timed_matmul(size: usize, f: F, name: let start = Instant::now(); f(&a[..], &b[..], &mut dest[..]); let dur = Instant::now() - start; - let nanos = dur.subsec_nanos() as u64 + dur.as_secs() * 1_000_000_000u64; + let nanos = u64::from(dur.subsec_nanos()) + dur.as_secs() * 1_000_000_000u64; println!( "{}:\t{}x{} matrix: {} s", name, @@ -398,7 +396,7 @@ fn timed_matmul(size: usize, f: F, name: size, nanos as f32 / 1e9f32 ); - return nanos; + nanos } pub fn main(args: &[String]) { diff --git a/rayon-demo/src/mergesort/mod.rs b/rayon-demo/src/mergesort/mod.rs index c93f7664a..27d94da22 100644 --- a/rayon-demo/src/mergesort/mod.rs +++ b/rayon-demo/src/mergesort/mod.rs @@ -1,7 +1,7 @@ use rand::distributions::Standard; use rand::Rng; -const USAGE: &'static str = " +const USAGE: &str = " Usage: mergesort bench [--size N] mergesort --help @@ -213,7 +213,7 @@ pub fn is_sorted(v: &mut [T]) -> bool { } let (a, b) = v.split_at_mut(mid); let (left, right) = rayon::join(|| is_sorted(a), || is_sorted(b)); - return left && right; + left && right } fn default_vec(n: usize) -> Vec { @@ -227,13 +227,13 @@ fn timed_sort(n: usize, f: F, name: &str) -> u64 { let start = Instant::now(); f(&mut v[..]); let dur = Instant::now() - start; - let nanos = dur.subsec_nanos() as u64 + dur.as_secs() * 1_000_000_000u64; + let nanos = u64::from(dur.subsec_nanos()) + dur.as_secs() * 1_000_000_000u64; println!("{}: sorted {} ints: {} s", name, n, nanos as f32 / 1e9f32); // Check correctness assert!(is_sorted(&mut v[..])); - return nanos; + nanos } pub fn main(args: &[String]) { diff --git a/rayon-demo/src/nbody/mod.rs b/rayon-demo/src/nbody/mod.rs index 9b7791812..99954ad51 100644 --- a/rayon-demo/src/nbody/mod.rs +++ b/rayon-demo/src/nbody/mod.rs @@ -8,7 +8,7 @@ mod visualize; use self::nbody::NBodyBenchmark; use self::visualize::visualize_benchmarks; -const USAGE: &'static str = " +const USAGE: &str = " Usage: nbody bench [--mode MODE --bodies N --ticks N] nbody visualize [--mode MODE --bodies N] nbody --help diff --git a/rayon-demo/src/nbody/nbody.rs b/rayon-demo/src/nbody/nbody.rs index 768c54306..6bdcc2a91 100644 --- a/rayon-demo/src/nbody/nbody.rs +++ b/rayon-demo/src/nbody/nbody.rs @@ -76,9 +76,9 @@ impl NBodyBenchmark { }; Body { - position: position, - velocity: velocity, - velocity2: velocity2, + position, + velocity, + velocity2, } }) .collect(); diff --git a/rayon-demo/src/pythagoras/mod.rs b/rayon-demo/src/pythagoras/mod.rs index 53faeca65..980f12ade 100644 --- a/rayon-demo/src/pythagoras/mod.rs +++ b/rayon-demo/src/pythagoras/mod.rs @@ -23,11 +23,11 @@ where FN: Fn(Iter) -> N + Sync, N: ParallelIterator, { - map_m((1u32..2000).into_par_iter()) + map_m((1u32..2_000).into_par_iter()) .map(|m| -> u32 { map_n((1..m).into_par_iter()) .filter(|n| (m - n).is_odd() && m.gcd(n) == 1) - .map(|n| 4000000 / (m * m + n * n)) + .map(|n| 4_000_000 / (m * m + n * n)) .sum() }) .sum() @@ -35,13 +35,13 @@ where /// Same as par_euclid, without tweaking split lengths fn par_euclid_weightless() -> u32 { - (1u32..2000) + (1u32..2_000) .into_par_iter() .map(|m| -> u32 { (1..m) .into_par_iter() .filter(|n| (m - n).is_odd() && m.gcd(n) == 1) - .map(|n| 4000000 / (m * m + n * n)) + .map(|n| 4_000_000 / (m * m + n * n)) .sum() }) .sum() @@ -49,11 +49,11 @@ fn par_euclid_weightless() -> u32 { /// Same as par_euclid, without using rayon. fn euclid() -> u32 { - (1u32..2000) + (1u32..2_000) .map(|m| { (1..m) .filter(|n| (m - n).is_odd() && m.gcd(n) == 1) - .map(|n| 4000000 / (m * m + n * n)) + .map(|n| 4_000_000 / (m * m + n * n)) .fold(0, Add::add) }) .fold(0, Add::add) diff --git a/rayon-demo/src/quicksort/bench.rs b/rayon-demo/src/quicksort/bench.rs index e7ae3ed7b..70620ec1c 100644 --- a/rayon-demo/src/quicksort/bench.rs +++ b/rayon-demo/src/quicksort/bench.rs @@ -13,7 +13,7 @@ fn bench_harness(mut f: F, b: &mut test::Bencher) { sort_vec = base_vec.clone(); f(&mut sort_vec); }); - assert!(super::is_sorted(&mut sort_vec)); + assert!(super::is_sorted(&sort_vec)); } #[bench] diff --git a/rayon-demo/src/quicksort/mod.rs b/rayon-demo/src/quicksort/mod.rs index 0dd190e4c..bd362825b 100644 --- a/rayon-demo/src/quicksort/mod.rs +++ b/rayon-demo/src/quicksort/mod.rs @@ -1,6 +1,6 @@ #![allow(non_camel_case_types)] -const USAGE: &'static str = " +const USAGE: &str = " Usage: quicksort bench [options] quicksort --help @@ -118,13 +118,13 @@ fn timed_sort(n: usize, f: F, name: &str) -> u64 { let start = Instant::now(); f(&mut v[..]); let dur = Instant::now() - start; - let nanos = dur.subsec_nanos() as u64 + dur.as_secs() * 1_000_000_000u64; + let nanos = u64::from(dur.subsec_nanos()) + dur.as_secs() * 1_000_000_000u64; println!("{}: sorted {} ints: {} s", name, n, nanos as f32 / 1e9f32); // Check correctness - assert!(is_sorted(&mut v[..])); + assert!(is_sorted(&v[..])); - return nanos; + nanos } pub fn main(args: &[String]) { diff --git a/rayon-demo/src/sieve/mod.rs b/rayon-demo/src/sieve/mod.rs index e16d611f0..5ea1f6297 100644 --- a/rayon-demo/src/sieve/mod.rs +++ b/rayon-demo/src/sieve/mod.rs @@ -1,4 +1,4 @@ -const USAGE: &'static str = " +const USAGE: &str = " Usage: sieve bench sieve --help @@ -48,28 +48,28 @@ const CHUNK_SIZE: usize = 100_000; // Number of Primes < 10^n // https://oeis.org/A006880 -const NUM_PRIMES: &'static [usize] = &[ +const NUM_PRIMES: &[usize] = &[ 0, // primes in 0..10^0 4, // primes in 0..10^1 25, // etc 168, - 1229, - 9592, - 78498, - 664579, - 5761455, - 50847534, - 455052511, - 4118054813, - 37607912018, - 346065536839, - 3204941750802, - 29844570422669, - 279238341033925, - 2623557157654233, - 24739954287740860, - 234057667276344607, - 2220819602560918840, + 1_229, + 9_592, + 78_498, + 664_579, + 5_761_455, + 50_847_534, + 455_052_511, + 4_118_054_813, + 37_607_912_018, + 346_065_536_839, + 3_204_941_750_802, + 29_844_570_422_669, + 279_238_341_033_925, + 2_623_557_157_654_233, + 24_739_954_287_740_860, + 234_057_667_276_344_607, + 2_220_819_602_560_918_840, ]; // For all of these sieves, sieve[i]==true -> 2*i+1 is prime @@ -149,7 +149,7 @@ fn update_chunk(low: &[bool], chunk: &mut [bool], base: usize) { let pm = if pp < base { // p² is too small - find the first odd multiple that's in range - ((base + p - 1) / p | 1) * p + (((base + p - 1) / p) | 1) * p } else { pp }; diff --git a/rayon-demo/src/tsp/graph.rs b/rayon-demo/src/tsp/graph.rs index d3dafc2f1..2f90fb91e 100644 --- a/rayon-demo/src/tsp/graph.rs +++ b/rayon-demo/src/tsp/graph.rs @@ -15,7 +15,7 @@ pub struct Graph { impl Graph { pub fn new(num_nodes: usize) -> Graph { Graph { - num_nodes: num_nodes, + num_nodes, weights: iter::repeat(Weight::max()) .take(num_nodes * num_nodes) .collect(), @@ -58,9 +58,9 @@ impl Graph { pub fn edges<'a>(&'a self, source: Node) -> impl Iterator + 'a { self.all_nodes().filter_map(move |target| { self.edge_weight(source, target).map(|weight| Edge { - source: source, - target: target, - weight: weight, + source, + target, + weight, }) }) } @@ -73,10 +73,10 @@ pub struct Node { impl Node { pub fn new(index: usize) -> Node { - Node { index: index } + Node { index } } - pub fn index(&self) -> usize { + pub fn index(self) -> usize { self.index } } diff --git a/rayon-demo/src/tsp/mod.rs b/rayon-demo/src/tsp/mod.rs index 2318108ae..d29bdc087 100644 --- a/rayon-demo/src/tsp/mod.rs +++ b/rayon-demo/src/tsp/mod.rs @@ -22,7 +22,7 @@ mod weight; use self::graph::{Graph, Node}; use self::solver::SolverCx; -const USAGE: &'static str = " +const USAGE: &str = " Usage: tsp bench [--seq-threshold N] [--from N] Parallel traveling salesman problem solver. Data input is expected to @@ -91,7 +91,7 @@ fn run_solver(datafile: &Path, seq_threshold: usize, from: usize) -> Result<(), println!("Total search time: {:?}", par_time); if let Some(path) = path { println!("Cheapest path cost: {}", weight.to_usize()); - let mut output = format!("Cheapest path:"); + let mut output = "Cheapest path:".to_string(); for node in path { output.push_str(&format!(" {}", node.index())); } diff --git a/rayon-demo/src/tsp/parser.rs b/rayon-demo/src/tsp/parser.rs index 2322c2969..e8cf15bb8 100644 --- a/rayon-demo/src/tsp/parser.rs +++ b/rayon-demo/src/tsp/parser.rs @@ -151,7 +151,7 @@ impl<'text> Data<'text> { let mut lines = data.lines(); let current_line = lines.next(); Data { - current_line: current_line, + current_line, line_num: 1, next_lines: lines, } diff --git a/rayon-demo/src/tsp/solver.rs b/rayon-demo/src/tsp/solver.rs index 37dd6f1c0..b461b3a22 100644 --- a/rayon-demo/src/tsp/solver.rs +++ b/rayon-demo/src/tsp/solver.rs @@ -29,8 +29,8 @@ pub struct TourId { impl<'s> SolverCx<'s> { pub fn new(graph: &'s Graph, seq_threshold: usize) -> Self { SolverCx { - graph: graph, - seq_threshold: seq_threshold, + graph, + seq_threshold, priority_queue: Mutex::new(BinaryHeap::new()), tour_counter: AtomicUsize::new(0), min_tour_weight: AtomicUsize::new(usize::MAX), @@ -47,12 +47,12 @@ impl<'s> SolverCx<'s> { .get_mut() .unwrap() .push(Arc::new(TourPrefix { - id: id, - node: node, + id, + node, len: 1, prefix_weight: Weight::zero(), priority: Weight::max().to_priority(), - visited: visited, + visited, previous: None, })); @@ -80,8 +80,7 @@ impl<'s> SolverCx<'s> { pub fn dequeue(&self) -> Option> { let mut priority_queue = self.priority_queue.lock().unwrap(); - let element = priority_queue.pop(); - element + priority_queue.pop() } pub fn min_tour_weight(&self) -> Weight { @@ -91,12 +90,12 @@ impl<'s> SolverCx<'s> { Weight::new(self.min_tour_weight.load(Ordering::Relaxed)) } - pub fn add_complete_tour(&self, tour: &Vec, weight: Weight) { + pub fn add_complete_tour(&self, tour: &[Node], weight: Weight) { if weight < self.min_tour_weight() { let mut min_tour = self.min_tour.lock().unwrap(); if min_tour.is_none() || weight < self.min_tour_weight() { // this is a new minimum! - *min_tour = Some(tour.clone()); + *min_tour = Some(tour.to_vec()); self.min_tour_weight .store(weight.to_usize(), Ordering::Relaxed); } diff --git a/rayon-demo/src/tsp/step.rs b/rayon-demo/src/tsp/step.rs index 8c13d3201..311c55244 100644 --- a/rayon-demo/src/tsp/step.rs +++ b/rayon-demo/src/tsp/step.rs @@ -41,8 +41,8 @@ fn split_tour<'s>(scope: &Scope<'s>, solver: &'s SolverCx<'s>, element: Arc + Send, { let list: LinkedList> = pi - .fold( - || Vec::new(), - |mut vec, elem| { - vec.push(elem); - vec - }, - ) + .fold(Vec::new, |mut vec, elem| { + vec.push(elem); + vec + }) .collect(); list.into_iter().fold(Vec::new(), |mut vec, mut sub| { vec.append(&mut sub); @@ -41,13 +38,10 @@ mod util { PI: ParallelIterator + Send, { let list: LinkedList> = pi - .fold( - || Vec::new(), - |mut vec, elem| { - vec.push(elem); - vec - }, - ) + .fold(Vec::new, |mut vec, elem| { + vec.push(elem); + vec + }) .collect(); let len = list.iter().map(Vec::len).sum(); @@ -65,13 +59,10 @@ mod util { PI: ParallelIterator + Send, { let list: LinkedList> = pi - .fold( - || Vec::new(), - |mut vec, elem| { - vec.push(elem); - vec - }, - ) + .fold(Vec::new, |mut vec, elem| { + vec.push(elem); + vec + }) .map(|vec| { let mut list = LinkedList::new(); list.push_back(vec); @@ -97,21 +88,15 @@ mod util { PI: ParallelIterator + Send, { let vecs: Vec> = pi - .fold( - || Vec::new(), - |mut vec, elem| { - vec.push(elem); - vec - }, - ) + .fold(Vec::new, |mut vec, elem| { + vec.push(elem); + vec + }) .map(|v| vec![v]) - .reduce( - || Vec::new(), - |mut left, mut right| { - left.append(&mut right); - left - }, - ); + .reduce(Vec::new, |mut left, mut right| { + left.append(&mut right); + left + }); let len = vecs.iter().map(Vec::len).sum(); vecs.into_iter() @@ -127,20 +112,14 @@ mod util { T: Send, PI: ParallelIterator + Send, { - pi.fold( - || Vec::new(), - |mut vec, x| { - vec.push(x); - vec - }, - ) - .reduce( - || Vec::new(), - |mut vec1, mut vec2| { - vec1.append(&mut vec2); - vec1 - }, - ) + pi.fold(Vec::new, |mut vec, x| { + vec.push(x); + vec + }) + .reduce(Vec::new, |mut vec1, mut vec2| { + vec1.append(&mut vec2); + vec1 + }) } } @@ -206,7 +185,7 @@ mod vec_i { (0_u32..N).into_par_iter() } - fn check(v: &Vec) { + fn check(v: &[u32]) { assert!(v.iter().cloned().eq(0..N)); } @@ -242,7 +221,7 @@ mod vec_i_filtered { (0_u32..N).into_par_iter().filter(|_| true) } - fn check(v: &Vec) { + fn check(v: &[u32]) { assert!(v.iter().cloned().eq(0..N)); } diff --git a/src/iter/chain.rs b/src/iter/chain.rs index 29f43d926..98691aae9 100644 --- a/src/iter/chain.rs +++ b/src/iter/chain.rs @@ -20,15 +20,15 @@ where b: B, } -/// Create a new `Chain` iterator. -/// -/// NB: a free fn because it is NOT part of the end-user API. -pub fn new(a: A, b: B) -> Chain +impl Chain where A: ParallelIterator, B: ParallelIterator, { - Chain { a: a, b: b } + /// Create a new `Chain` iterator. + pub(super) fn new(a: A, b: B) -> Self { + Chain { a, b } + } } impl ParallelIterator for Chain @@ -60,10 +60,7 @@ where } fn opt_len(&self) -> Option { - match (self.a.opt_len(), self.b.opt_len()) { - (Some(a_len), Some(b_len)) => a_len.checked_add(b_len), - _ => None, - } + self.a.opt_len()?.checked_add(self.b.opt_len()?) } } @@ -92,8 +89,8 @@ where { let a_len = self.a.len(); return self.a.with_producer(CallbackA { - callback: callback, - a_len: a_len, + callback, + a_len, b: self.b, }); @@ -114,11 +111,11 @@ where where A: Producer, { - return self.b.with_producer(CallbackB { + self.b.with_producer(CallbackB { callback: self.callback, a_len: self.a_len, - a_producer: a_producer, - }); + a_producer, + }) } } @@ -164,11 +161,7 @@ where B: Producer, { fn new(a_len: usize, a: A, b: B) -> Self { - ChainProducer { - a_len: a_len, - a: a, - b: b, - } + ChainProducer { a_len, a, b } } } diff --git a/src/iter/chunks.rs b/src/iter/chunks.rs index 9ac70c1ea..330a49dcb 100644 --- a/src/iter/chunks.rs +++ b/src/iter/chunks.rs @@ -20,14 +20,14 @@ where i: I, } -/// Create a new `Chunks` iterator -/// -/// NB: a free fn because it is NOT part of the end-user API. -pub fn new(i: I, size: usize) -> Chunks +impl Chunks where I: IndexedParallelIterator, { - Chunks { i: i, size: size } + /// Create a new `Chunks` iterator + pub(super) fn new(i: I, size: usize) -> Self { + Chunks { i, size } + } } impl ParallelIterator for Chunks @@ -70,8 +70,8 @@ where let len = self.i.len(); return self.i.with_producer(Callback { size: self.size, - len: len, - callback: callback, + len, + callback, }); struct Callback { @@ -93,7 +93,7 @@ where self.callback.callback(ChunkProducer { chunk_size: self.size, len: self.len, - base: base, + base, }) } } @@ -163,20 +163,16 @@ where type Item = Vec; fn next(&mut self) -> Option { - match self.inner.take() { - Some(producer) => { - if self.len > self.chunk_size { - let (left, right) = producer.split_at(self.chunk_size); - self.inner = Some(right); - self.len -= self.chunk_size; - Some(left.into_iter().collect()) - } else { - debug_assert!(self.len > 0); - self.len = 0; - Some(producer.into_iter().collect()) - } - } - _ => None, + let producer = self.inner.take()?; + if self.len > self.chunk_size { + let (left, right) = producer.split_at(self.chunk_size); + self.inner = Some(right); + self.len -= self.chunk_size; + Some(left.into_iter().collect()) + } else { + debug_assert!(self.len > 0); + self.len = 0; + Some(producer.into_iter().collect()) } } @@ -201,24 +197,20 @@ where P: Producer, { fn next_back(&mut self) -> Option { - match self.inner.take() { - Some(producer) => { - if self.len > self.chunk_size { - let mut size = self.len % self.chunk_size; - if size == 0 { - size = self.chunk_size; - } - let (left, right) = producer.split_at(self.len - size); - self.inner = Some(left); - self.len -= size; - Some(right.into_iter().collect()) - } else { - debug_assert!(self.len > 0); - self.len = 0; - Some(producer.into_iter().collect()) - } + let producer = self.inner.take()?; + if self.len > self.chunk_size { + let mut size = self.len % self.chunk_size; + if size == 0 { + size = self.chunk_size; } - _ => None, + let (left, right) = producer.split_at(self.len - size); + self.inner = Some(left); + self.len -= size; + Some(right.into_iter().collect()) + } else { + debug_assert!(self.len > 0); + self.len = 0; + Some(producer.into_iter().collect()) } } } diff --git a/src/iter/cloned.rs b/src/iter/cloned.rs index b3aff3c7f..2f4ec5d98 100644 --- a/src/iter/cloned.rs +++ b/src/iter/cloned.rs @@ -15,14 +15,14 @@ pub struct Cloned { base: I, } -/// Create a new `Cloned` iterator. -/// -/// NB: a free fn because it is NOT part of the end-user API. -pub fn new(base: I) -> Cloned +impl Cloned where I: ParallelIterator, { - Cloned { base: base } + /// Create a new `Cloned` iterator. + pub(super) fn new(base: I) -> Self { + Cloned { base } + } } impl<'a, T, I> ParallelIterator for Cloned @@ -66,7 +66,7 @@ where where CB: ProducerCallback, { - return self.base.with_producer(Callback { callback: callback }); + return self.base.with_producer(Callback { callback }); struct Callback { callback: CB, @@ -83,7 +83,7 @@ where where P: Producer, { - let producer = ClonedProducer { base: base }; + let producer = ClonedProducer { base }; self.callback.callback(producer) } } @@ -141,7 +141,7 @@ struct ClonedConsumer { impl ClonedConsumer { fn new(base: C) -> Self { - ClonedConsumer { base: base } + ClonedConsumer { base } } } diff --git a/src/iter/collect/consumer.rs b/src/iter/collect/consumer.rs index 74674854a..eafa72116 100644 --- a/src/iter/collect/consumer.rs +++ b/src/iter/collect/consumer.rs @@ -4,7 +4,7 @@ use std::ptr; use std::slice; use std::sync::atomic::{AtomicUsize, Ordering}; -pub struct CollectConsumer<'c, T: Send + 'c> { +pub(super) struct CollectConsumer<'c, T: Send + 'c> { /// Tracks how many items we successfully wrote. Used to guarantee /// safety in the face of panics or buggy parallel iterators. /// @@ -19,7 +19,7 @@ pub struct CollectConsumer<'c, T: Send + 'c> { target: &'c mut [T], } -pub struct CollectFolder<'c, T: Send + 'c> { +pub(super) struct CollectFolder<'c, T: Send + 'c> { global_writes: &'c AtomicUsize, local_writes: usize, @@ -30,11 +30,8 @@ pub struct CollectFolder<'c, T: Send + 'c> { impl<'c, T: Send + 'c> CollectConsumer<'c, T> { /// The target memory is considered uninitialized, and will be /// overwritten without dropping anything. - pub fn new(writes: &'c AtomicUsize, target: &'c mut [T]) -> CollectConsumer<'c, T> { - CollectConsumer { - writes: writes, - target: target, - } + pub(super) fn new(writes: &'c AtomicUsize, target: &'c mut [T]) -> Self { + CollectConsumer { writes, target } } } @@ -63,7 +60,7 @@ impl<'c, T: Send + 'c> Consumer for CollectConsumer<'c, T> { CollectFolder { global_writes: self.writes, local_writes: 0, - target: self.target.into_iter(), + target: self.target.iter_mut(), } } diff --git a/src/iter/collect/mod.rs b/src/iter/collect/mod.rs index 2a8b755ea..12be9f122 100644 --- a/src/iter/collect/mod.rs +++ b/src/iter/collect/mod.rs @@ -11,8 +11,8 @@ mod test; /// Collects the results of the exact iterator into the specified vector. /// -/// This is not directly public, but called by `IndexedParallelIterator::collect_into_vec`. -pub fn collect_into_vec(pi: I, v: &mut Vec) +/// This is called by `IndexedParallelIterator::collect_into_vec`. +pub(super) fn collect_into_vec(pi: I, v: &mut Vec) where I: IndexedParallelIterator, T: Send, @@ -46,8 +46,8 @@ where /// Unzips the results of the exact iterator into the specified vectors. /// -/// This is not directly public, but called by `IndexedParallelIterator::unzip_into_vecs`. -pub fn unzip_into_vecs(pi: I, left: &mut Vec, right: &mut Vec) +/// This is called by `IndexedParallelIterator::unzip_into_vecs`. +pub(super) fn unzip_into_vecs(pi: I, left: &mut Vec, right: &mut Vec) where I: IndexedParallelIterator, A: Send, @@ -78,13 +78,13 @@ impl<'c, T: Send + 'c> Collect<'c, T> { fn new(vec: &'c mut Vec, len: usize) -> Self { Collect { writes: AtomicUsize::new(0), - vec: vec, - len: len, + vec, + len, } } /// Create a consumer on a slice of our memory. - fn as_consumer(&mut self) -> CollectConsumer { + fn as_consumer(&mut self) -> CollectConsumer<'_, T> { // Reserve the new space. self.vec.reserve(self.len); diff --git a/src/iter/empty.rs b/src/iter/empty.rs index 249fcdba3..2f68df46c 100644 --- a/src/iter/empty.rs +++ b/src/iter/empty.rs @@ -40,7 +40,7 @@ impl Clone for Empty { } impl fmt::Debug for Empty { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.pad("Empty") } } diff --git a/src/iter/enumerate.rs b/src/iter/enumerate.rs index 794f0fbb5..5d6a388cd 100644 --- a/src/iter/enumerate.rs +++ b/src/iter/enumerate.rs @@ -15,14 +15,14 @@ pub struct Enumerate { base: I, } -/// Create a new `Enumerate` iterator. -/// -/// NB: a free fn because it is NOT part of the end-user API. -pub fn new(base: I) -> Enumerate +impl Enumerate where I: IndexedParallelIterator, { - Enumerate { base: base } + /// Create a new `Enumerate` iterator. + pub(super) fn new(base: I) -> Self { + Enumerate { base } + } } impl ParallelIterator for Enumerate @@ -59,7 +59,7 @@ where where CB: ProducerCallback, { - return self.base.with_producer(Callback { callback: callback }); + return self.base.with_producer(Callback { callback }); struct Callback { callback: CB, @@ -74,10 +74,7 @@ where where P: Producer, { - let producer = EnumerateProducer { - base: base, - offset: 0, - }; + let producer = EnumerateProducer { base, offset: 0 }; self.callback.callback(producer) } } diff --git a/src/iter/filter.rs b/src/iter/filter.rs index 4feb240e3..48dda5e51 100644 --- a/src/iter/filter.rs +++ b/src/iter/filter.rs @@ -16,21 +16,18 @@ pub struct Filter { } impl Debug for Filter { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("Filter").field("base", &self.base).finish() } } -/// Create a new `Filter` iterator. -/// -/// NB: a free fn because it is NOT part of the end-user API. -pub fn new(base: I, filter_op: P) -> Filter +impl Filter where I: ParallelIterator, { - Filter { - base: base, - filter_op: filter_op, + /// Create a new `Filter` iterator. + pub(super) fn new(base: I, filter_op: P) -> Self { + Filter { base, filter_op } } } @@ -60,10 +57,7 @@ struct FilterConsumer<'p, C, P: 'p> { impl<'p, C, P> FilterConsumer<'p, C, P> { fn new(base: C, filter_op: &'p P) -> Self { - FilterConsumer { - base: base, - filter_op: filter_op, - } + FilterConsumer { base, filter_op } } } @@ -127,10 +121,7 @@ where let filter_op = self.filter_op; if filter_op(&item) { let base = self.base.consume(item); - FilterFolder { - base: base, - filter_op: filter_op, - } + FilterFolder { base, filter_op } } else { self } diff --git a/src/iter/filter_map.rs b/src/iter/filter_map.rs index 42c4f3274..885998dce 100644 --- a/src/iter/filter_map.rs +++ b/src/iter/filter_map.rs @@ -16,23 +16,17 @@ pub struct FilterMap { } impl Debug for FilterMap { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("FilterMap") .field("base", &self.base) .finish() } } -/// Create a new `FilterMap` iterator. -/// -/// NB: a free fn because it is NOT part of the end-user API. -pub fn new(base: I, filter_op: P) -> FilterMap -where - I: ParallelIterator, -{ - FilterMap { - base: base, - filter_op: filter_op, +impl FilterMap { + /// Create a new `FilterMap` iterator. + pub(super) fn new(base: I, filter_op: P) -> Self { + FilterMap { base, filter_op } } } @@ -63,10 +57,7 @@ struct FilterMapConsumer<'p, C, P: 'p> { impl<'p, C, P: 'p> FilterMapConsumer<'p, C, P> { fn new(base: C, filter_op: &'p P) -> Self { - FilterMapConsumer { - base: base, - filter_op: filter_op, - } + FilterMapConsumer { base, filter_op } } } @@ -91,7 +82,7 @@ where fn into_folder(self) -> Self::Folder { let base = self.base.into_folder(); FilterMapFolder { - base: base, + base, filter_op: self.filter_op, } } @@ -131,10 +122,7 @@ where let filter_op = self.filter_op; if let Some(mapped_item) = filter_op(item) { let base = self.base.consume(mapped_item); - FilterMapFolder { - base: base, - filter_op: filter_op, - } + FilterMapFolder { base, filter_op } } else { self } diff --git a/src/iter/find.rs b/src/iter/find.rs index d64665b7c..86b1fc17c 100644 --- a/src/iter/find.rs +++ b/src/iter/find.rs @@ -2,7 +2,7 @@ use super::plumbing::*; use super::*; use std::sync::atomic::{AtomicBool, Ordering}; -pub fn find(pi: I, find_op: P) -> Option +pub(super) fn find(pi: I, find_op: P) -> Option where I: ParallelIterator, P: Fn(&I::Item) -> bool + Sync, @@ -19,10 +19,7 @@ struct FindConsumer<'p, P: 'p> { impl<'p, P> FindConsumer<'p, P> { fn new(find_op: &'p P, found: &'p AtomicBool) -> Self { - FindConsumer { - find_op: find_op, - found: found, - } + FindConsumer { find_op, found } } } diff --git a/src/iter/find_first_last/mod.rs b/src/iter/find_first_last/mod.rs index 81bd80072..b89aeec3f 100644 --- a/src/iter/find_first_last/mod.rs +++ b/src/iter/find_first_last/mod.rs @@ -38,7 +38,7 @@ fn better_position(pos1: usize, pos2: usize, mp: MatchPosition) -> bool { } } -pub fn find_first(pi: I, find_op: P) -> Option +pub(super) fn find_first(pi: I, find_op: P) -> Option where I: ParallelIterator, P: Fn(&I::Item) -> bool + Sync, @@ -48,7 +48,7 @@ where pi.drive_unindexed(consumer) } -pub fn find_last(pi: I, find_op: P) -> Option +pub(super) fn find_last(pi: I, find_op: P) -> Option where I: ParallelIterator, P: Fn(&I::Item) -> bool + Sync, @@ -69,11 +69,11 @@ struct FindConsumer<'p, P: 'p> { impl<'p, P> FindConsumer<'p, P> { fn new(find_op: &'p P, match_position: MatchPosition, best_found: &'p AtomicUsize) -> Self { FindConsumer { - find_op: find_op, + find_op, lower_bound: Cell::new(0), upper_bound: usize::max_value(), - match_position: match_position, - best_found: best_found, + match_position, + best_found, } } diff --git a/src/iter/flat_map.rs b/src/iter/flat_map.rs index 046656509..076c23ad5 100644 --- a/src/iter/flat_map.rs +++ b/src/iter/flat_map.rs @@ -16,21 +16,15 @@ pub struct FlatMap { } impl Debug for FlatMap { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("FlatMap").field("base", &self.base).finish() } } -/// Create a new `FlatMap` iterator. -/// -/// NB: a free fn because it is NOT part of the end-user API. -pub fn new(base: I, map_op: F) -> FlatMap -where - I: ParallelIterator, -{ - FlatMap { - base: base, - map_op: map_op, +impl FlatMap { + /// Create a new `FlatMap` iterator. + pub(super) fn new(base: I, map_op: F) -> Self { + FlatMap { base, map_op } } } @@ -64,10 +58,7 @@ struct FlatMapConsumer<'f, C, F: 'f> { impl<'f, C, F> FlatMapConsumer<'f, C, F> { fn new(base: C, map_op: &'f F) -> Self { - FlatMapConsumer { - base: base, - map_op: map_op, - } + FlatMapConsumer { base, map_op } } } @@ -149,8 +140,8 @@ where FlatMapFolder { base: self.base, - map_op: map_op, - previous: previous, + map_op, + previous, } } diff --git a/src/iter/flatten.rs b/src/iter/flatten.rs index 3534b9168..5862adc81 100644 --- a/src/iter/flatten.rs +++ b/src/iter/flatten.rs @@ -13,15 +13,15 @@ pub struct Flatten { base: I, } -/// Create a new `Flatten` iterator. -/// -/// NB: Not part of the public API. -pub fn new(base: I) -> Flatten +impl Flatten where I: ParallelIterator, PI: IntoParallelIterator + Send, { - Flatten { base: base } + /// Create a new `Flatten` iterator. + pub(super) fn new(base: I) -> Self { + Flatten { base } + } } impl ParallelIterator for Flatten diff --git a/src/iter/fold.rs b/src/iter/fold.rs index 0b4b12ba8..9ae937c9d 100644 --- a/src/iter/fold.rs +++ b/src/iter/fold.rs @@ -3,17 +3,19 @@ use super::*; use std::fmt::{self, Debug}; -pub fn fold(base: I, identity: ID, fold_op: F) -> Fold +impl Fold where I: ParallelIterator, F: Fn(U, I::Item) -> U + Sync + Send, ID: Fn() -> U + Sync + Send, U: Send, { - Fold { - base: base, - identity: identity, - fold_op: fold_op, + pub(super) fn new(base: I, identity: ID, fold_op: F) -> Self { + Fold { + base, + identity, + fold_op, + } } } @@ -31,7 +33,7 @@ pub struct Fold { } impl Debug for Fold { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("Fold").field("base", &self.base).finish() } } @@ -137,7 +139,7 @@ where FoldFolder { base: self.base, fold_op: self.fold_op, - item: item, + item, } } @@ -153,8 +155,8 @@ where .fold(self.item, self.fold_op); FoldFolder { - base: base, - item: item, + base, + item, fold_op: self.fold_op, } } @@ -170,16 +172,18 @@ where // /////////////////////////////////////////////////////////////////////////// -pub fn fold_with(base: I, item: U, fold_op: F) -> FoldWith +impl FoldWith where I: ParallelIterator, F: Fn(U, I::Item) -> U + Sync + Send, U: Send + Clone, { - FoldWith { - base: base, - item: item, - fold_op: fold_op, + pub(super) fn new(base: I, item: U, fold_op: F) -> Self { + FoldWith { + base, + item, + fold_op, + } } } @@ -197,7 +201,7 @@ pub struct FoldWith { } impl Debug for FoldWith { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("FoldWith") .field("base", &self.base) .field("item", &self.item) diff --git a/src/iter/for_each.rs b/src/iter/for_each.rs index d9681dfca..994140458 100644 --- a/src/iter/for_each.rs +++ b/src/iter/for_each.rs @@ -2,13 +2,13 @@ use super::noop::*; use super::plumbing::*; use super::ParallelIterator; -pub fn for_each(pi: I, op: &F) +pub(super) fn for_each(pi: I, op: &F) where I: ParallelIterator, F: Fn(T) + Sync, T: Send, { - let consumer = ForEachConsumer { op: op }; + let consumer = ForEachConsumer { op }; pi.drive_unindexed(consumer) } diff --git a/src/iter/inspect.rs b/src/iter/inspect.rs index d140f4b94..145032d01 100644 --- a/src/iter/inspect.rs +++ b/src/iter/inspect.rs @@ -19,21 +19,18 @@ pub struct Inspect { } impl Debug for Inspect { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("Inspect").field("base", &self.base).finish() } } -/// Create a new `Inspect` iterator. -/// -/// NB: a free fn because it is NOT part of the end-user API. -pub fn new(base: I, inspect_op: F) -> Inspect +impl Inspect where I: ParallelIterator, { - Inspect { - base: base, - inspect_op: inspect_op, + /// Create a new `Inspect` iterator. + pub(super) fn new(base: I, inspect_op: F) -> Self { + Inspect { base, inspect_op } } } @@ -79,7 +76,7 @@ where CB: ProducerCallback, { return self.base.with_producer(Callback { - callback: callback, + callback, inspect_op: self.inspect_op, }); @@ -100,7 +97,7 @@ where P: Producer, { let producer = InspectProducer { - base: base, + base, inspect_op: &self.inspect_op, }; self.callback.callback(producer) @@ -172,10 +169,7 @@ struct InspectConsumer<'f, C, F: 'f> { impl<'f, C, F> InspectConsumer<'f, C, F> { fn new(base: C, inspect_op: &'f F) -> Self { - InspectConsumer { - base: base, - inspect_op: inspect_op, - } + InspectConsumer { base, inspect_op } } } diff --git a/src/iter/interleave.rs b/src/iter/interleave.rs index 4e6bbd182..028f70b8d 100644 --- a/src/iter/interleave.rs +++ b/src/iter/interleave.rs @@ -20,15 +20,15 @@ where j: J, } -/// Create a new `Interleave` iterator -/// -/// NB: a free fn because it is NOT part of the end-user API. -pub fn new(i: I, j: J) -> Interleave +impl Interleave where I: IndexedParallelIterator, J: IndexedParallelIterator, { - Interleave { i: i, j: j } + /// Create a new `Interleave` iterator + pub(super) fn new(i: I, j: J) -> Self { + Interleave { i, j } + } } impl ParallelIterator for Interleave @@ -72,9 +72,9 @@ where { let (i_len, j_len) = (self.i.len(), self.j.len()); return self.i.with_producer(CallbackI { - callback: callback, - i_len: i_len, - j_len: j_len, + callback, + i_len, + j_len, i_next: false, j: self.j, }); @@ -99,7 +99,7 @@ where I: Producer, { self.j.with_producer(CallbackJ { - i_producer: i_producer, + i_producer, i_len: self.i_len, j_len: self.j_len, i_next: self.i_next, @@ -140,7 +140,7 @@ where } } -pub struct InterleaveProducer +struct InterleaveProducer where I: Producer, J: Producer, @@ -159,11 +159,11 @@ where { fn new(i: I, j: J, i_len: usize, j_len: usize, i_next: bool) -> InterleaveProducer { InterleaveProducer { - i: i, - j: j, - i_len: i_len, - j_len: j_len, - i_next: i_next, + i, + j, + i_len, + j_len, + i_next, } } } @@ -247,7 +247,7 @@ where /// ExactSizeIterator. /// /// This iterator is fused. -pub struct InterleaveSeq { +struct InterleaveSeq { i: Fuse, j: Fuse, diff --git a/src/iter/interleave_shortest.rs b/src/iter/interleave_shortest.rs index b1772d281..9b8f1a9e4 100644 --- a/src/iter/interleave_shortest.rs +++ b/src/iter/interleave_shortest.rs @@ -20,24 +20,24 @@ where interleave: Interleave, Take>, } -/// Create a new `InterleaveShortest` iterator -/// -/// NB: a free fn because it is NOT part of the end-user API. -pub fn new(i: I, j: J) -> InterleaveShortest +impl InterleaveShortest where I: IndexedParallelIterator, J: IndexedParallelIterator, { - InterleaveShortest { - interleave: if i.len() <= j.len() { - // take equal lengths from both iterators - let n = i.len(); - i.take(n).interleave(j.take(n)) - } else { - // take one extra item from the first iterator - let n = j.len(); - i.take(n + 1).interleave(j.take(n)) - }, + /// Create a new `InterleaveShortest` iterator + pub(super) fn new(i: I, j: J) -> Self { + InterleaveShortest { + interleave: if i.len() <= j.len() { + // take equal lengths from both iterators + let n = i.len(); + i.take(n).interleave(j.take(n)) + } else { + // take one extra item from the first iterator + let n = j.len(); + i.take(n + 1).interleave(j.take(n)) + }, + } } } diff --git a/src/iter/intersperse.rs b/src/iter/intersperse.rs index 133dd8cf9..ba6e515b8 100644 --- a/src/iter/intersperse.rs +++ b/src/iter/intersperse.rs @@ -20,17 +20,14 @@ where item: I::Item, } -/// Create a new `Intersperse` iterator -/// -/// NB: a free fn because it is NOT part of the end-user API. -pub fn new(base: I, item: I::Item) -> Intersperse +impl Intersperse where I: ParallelIterator, I::Item: Clone, { - Intersperse { - base: base, - item: item, + /// Create a new `Intersperse` iterator + pub(super) fn new(base: I, item: I::Item) -> Self { + Intersperse { base, item } } } @@ -50,10 +47,9 @@ where } fn opt_len(&self) -> Option { - match self.base.opt_len() { - None => None, - Some(0) => Some(0), - Some(len) => len.checked_add(len - 1), + match self.base.opt_len()? { + 0 => Some(0), + len => len.checked_add(len - 1), } } } @@ -86,9 +82,9 @@ where { let len = self.len(); return self.base.with_producer(Callback { - callback: callback, + callback, item: self.item, - len: len, + len, }); struct Callback { @@ -131,9 +127,9 @@ where { fn new(base: P, item: P::Item, len: usize) -> Self { IntersperseProducer { - base: base, - item: item, - len: len, + base, + item, + len, clone_first: false, } } @@ -292,8 +288,8 @@ where { fn new(base: C, item: T) -> Self { IntersperseConsumer { - base: base, - item: item, + base, + item, clone_first: false.into(), } } @@ -398,9 +394,9 @@ where first.into_iter().chain(iter::once(item)) })); IntersperseFolder { - base: base, + base, item: between_item, - clone_first: clone_first, + clone_first, } } diff --git a/src/iter/len.rs b/src/iter/len.rs index 24582ce97..5db0002f1 100644 --- a/src/iter/len.rs +++ b/src/iter/len.rs @@ -14,16 +14,13 @@ pub struct MinLen { min: usize, } -/// Create a new `MinLen` iterator. -/// -/// NB: a free fn because it is NOT part of the end-user API. -pub fn new_min_len(base: I, min: usize) -> MinLen +impl MinLen where I: IndexedParallelIterator, { - MinLen { - base: base, - min: min, + /// Create a new `MinLen` iterator. + pub(super) fn new(base: I, min: usize) -> Self { + MinLen { base, min } } } @@ -62,7 +59,7 @@ where CB: ProducerCallback, { return self.base.with_producer(Callback { - callback: callback, + callback, min: self.min, }); @@ -81,7 +78,7 @@ where P: Producer, { let producer = MinLenProducer { - base: base, + base, min: self.min, }; self.callback.callback(producer) @@ -151,16 +148,13 @@ pub struct MaxLen { max: usize, } -/// Create a new `MaxLen` iterator. -/// -/// NB: a free fn because it is NOT part of the end-user API. -pub fn new_max_len(base: I, max: usize) -> MaxLen +impl MaxLen where I: IndexedParallelIterator, { - MaxLen { - base: base, - max: max, + /// Create a new `MaxLen` iterator. + pub(super) fn new(base: I, max: usize) -> Self { + MaxLen { base, max } } } @@ -199,7 +193,7 @@ where CB: ProducerCallback, { return self.base.with_producer(Callback { - callback: callback, + callback, max: self.max, }); @@ -218,7 +212,7 @@ where P: Producer, { let producer = MaxLenProducer { - base: base, + base, max: self.max, }; self.callback.callback(producer) diff --git a/src/iter/map.rs b/src/iter/map.rs index e9d7e3d14..218539835 100644 --- a/src/iter/map.rs +++ b/src/iter/map.rs @@ -18,21 +18,18 @@ pub struct Map { } impl Debug for Map { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("Map").field("base", &self.base).finish() } } -/// Create a new `Map` iterator. -/// -/// NB: a free fn because it is NOT part of the end-user API. -pub fn new(base: I, map_op: F) -> Map +impl Map where I: ParallelIterator, { - Map { - base: base, - map_op: map_op, + /// Create a new `Map` iterator. + pub(super) fn new(base: I, map_op: F) -> Self { + Map { base, map_op } } } @@ -80,7 +77,7 @@ where CB: ProducerCallback, { return self.base.with_producer(Callback { - callback: callback, + callback, map_op: self.map_op, }); @@ -102,7 +99,7 @@ where P: Producer, { let producer = MapProducer { - base: base, + base, map_op: &self.map_op, }; self.callback.callback(producer) @@ -174,10 +171,7 @@ struct MapConsumer<'f, C, F: 'f> { impl<'f, C, F> MapConsumer<'f, C, F> { fn new(base: C, map_op: &'f F) -> Self { - MapConsumer { - base: base, - map_op: map_op, - } + MapConsumer { base, map_op } } } diff --git a/src/iter/map_with.rs b/src/iter/map_with.rs index 81a238e1c..c5eafc728 100644 --- a/src/iter/map_with.rs +++ b/src/iter/map_with.rs @@ -18,7 +18,7 @@ pub struct MapWith { } impl Debug for MapWith { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("MapWith") .field("base", &self.base) .field("item", &self.item) @@ -26,17 +26,13 @@ impl Debug for MapWith { } } -/// Create a new `MapWith` iterator. -/// -/// NB: a free fn because it is NOT part of the end-user API. -pub fn new(base: I, item: T, map_op: F) -> MapWith +impl MapWith where I: ParallelIterator, { - MapWith { - base: base, - item: item, - map_op: map_op, + /// Create a new `MapWith` iterator. + pub(super) fn new(base: I, item: T, map_op: F) -> Self { + MapWith { base, item, map_op } } } @@ -86,7 +82,7 @@ where CB: ProducerCallback, { return self.base.with_producer(Callback { - callback: callback, + callback, item: self.item, map_op: self.map_op, }); @@ -111,7 +107,7 @@ where P: Producer, { let producer = MapWithProducer { - base: base, + base, item: self.item, map_op: &self.map_op, }; @@ -198,9 +194,8 @@ where type Item = R; fn next(&mut self) -> Option { - self.base - .next() - .map(|item| (self.map_op)(&mut self.item, item)) + let item = self.base.next()?; + Some((self.map_op)(&mut self.item, item)) } fn size_hint(&self) -> (usize, Option) { @@ -215,9 +210,8 @@ where R: Send, { fn next_back(&mut self) -> Option { - self.base - .next_back() - .map(|item| (self.map_op)(&mut self.item, item)) + let item = self.base.next_back()?; + Some((self.map_op)(&mut self.item, item)) } } @@ -240,11 +234,7 @@ struct MapWithConsumer<'f, C, U, F: 'f> { impl<'f, C, U, F> MapWithConsumer<'f, C, U, F> { fn new(base: C, item: U, map_op: &'f F) -> Self { - MapWithConsumer { - base: base, - item: item, - map_op: map_op, - } + MapWithConsumer { base, item, map_op } } } @@ -355,22 +345,18 @@ pub struct MapInit { } impl Debug for MapInit { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("MapInit").field("base", &self.base).finish() } } -/// Create a new `MapInit` iterator. -/// -/// NB: a free fn because it is NOT part of the end-user API. -pub fn new_init(base: I, init: INIT, map_op: F) -> MapInit +impl MapInit where I: ParallelIterator, { - MapInit { - base: base, - init: init, - map_op: map_op, + /// Create a new `MapInit` iterator. + pub(super) fn new(base: I, init: INIT, map_op: F) -> Self { + MapInit { base, init, map_op } } } @@ -420,7 +406,7 @@ where CB: ProducerCallback, { return self.base.with_producer(Callback { - callback: callback, + callback, init: self.init, map_op: self.map_op, }); @@ -445,7 +431,7 @@ where P: Producer, { let producer = MapInitProducer { - base: base, + base, init: &self.init, map_op: &self.map_op, }; @@ -528,11 +514,7 @@ struct MapInitConsumer<'f, C, INIT: 'f, F: 'f> { impl<'f, C, INIT, F> MapInitConsumer<'f, C, INIT, F> { fn new(base: C, init: &'f INIT, map_op: &'f F) -> Self { - MapInitConsumer { - base: base, - init: init, - map_op: map_op, - } + MapInitConsumer { base, init, map_op } } } diff --git a/src/iter/mod.rs b/src/iter/mod.rs index 4ad49711b..425618437 100644 --- a/src/iter/mod.rs +++ b/src/iter/mod.rs @@ -558,7 +558,7 @@ pub trait ParallelIterator: Sized + Send { F: Fn(Self::Item) -> R + Sync + Send, R: Send, { - map::new(self, map_op) + Map::new(self, map_op) } /// Applies `map_op` to the given `init` value with each item of this @@ -596,7 +596,7 @@ pub trait ParallelIterator: Sized + Send { T: Send + Clone, R: Send, { - map_with::new(self, init, map_op) + MapWith::new(self, init, map_op) } /// Applies `map_op` to a value returned by `init` with each item of this @@ -636,7 +636,7 @@ pub trait ParallelIterator: Sized + Send { INIT: Fn() -> T + Sync + Send, R: Send, { - map_with::new_init(self, init, map_op) + MapInit::new(self, init, map_op) } /// Creates an iterator which clones all of its elements. This may be @@ -662,7 +662,7 @@ pub trait ParallelIterator: Sized + Send { T: 'a + Clone + Send, Self: ParallelIterator, { - cloned::new(self) + Cloned::new(self) } /// Applies `inspect_op` to a reference to each item of this iterator, @@ -698,7 +698,7 @@ pub trait ParallelIterator: Sized + Send { where OP: Fn(&Self::Item) + Sync + Send, { - inspect::new(self, inspect_op) + Inspect::new(self, inspect_op) } /// Mutates each item of this iterator before yielding it. @@ -718,7 +718,7 @@ pub trait ParallelIterator: Sized + Send { where F: Fn(&mut Self::Item) + Sync + Send, { - update::new(self, update_op) + Update::new(self, update_op) } /// Applies `filter_op` to each item of this iterator, producing a new @@ -739,7 +739,7 @@ pub trait ParallelIterator: Sized + Send { where P: Fn(&Self::Item) -> bool + Sync + Send, { - filter::new(self, filter_op) + Filter::new(self, filter_op) } /// Applies `filter_op` to each item of this iterator to get an `Option`, @@ -765,7 +765,7 @@ pub trait ParallelIterator: Sized + Send { P: Fn(Self::Item) -> Option + Sync + Send, R: Send, { - filter_map::new(self, filter_op) + FilterMap::new(self, filter_op) } /// Applies `map_op` to each item of this iterator to get nested iterators, @@ -789,7 +789,7 @@ pub trait ParallelIterator: Sized + Send { F: Fn(Self::Item) -> PI + Sync + Send, PI: IntoParallelIterator, { - flat_map::new(self, map_op) + FlatMap::new(self, map_op) } /// An adaptor that flattens iterable `Item`s into one large iterator @@ -808,7 +808,7 @@ pub trait ParallelIterator: Sized + Send { where Self::Item: IntoParallelIterator, { - flatten::new(self) + Flatten::new(self) } /// Reduces the items in the iterator into one item using `op`. @@ -1118,7 +1118,7 @@ pub trait ParallelIterator: Sized + Send { ID: Fn() -> T + Sync + Send, T: Send, { - fold::fold(self, identity, fold_op) + Fold::new(self, identity, fold_op) } /// Applies `fold_op` to the given `init` value with each item of this @@ -1145,7 +1145,7 @@ pub trait ParallelIterator: Sized + Send { F: Fn(T, Self::Item) -> T + Sync + Send, T: Send + Clone, { - fold::fold_with(self, init, fold_op) + FoldWith::new(self, init, fold_op) } /// Perform a fallible parallel fold. @@ -1179,7 +1179,7 @@ pub trait ParallelIterator: Sized + Send { ID: Fn() -> T + Sync + Send, R: Try + Send, { - try_fold::try_fold(self, identity, fold_op) + TryFold::new(self, identity, fold_op) } /// Perform a fallible parallel fold with a cloneable `init` value. @@ -1206,7 +1206,7 @@ pub trait ParallelIterator: Sized + Send { R: Try + Send, T: Clone + Send, { - try_fold::try_fold_with(self, init, fold_op) + TryFoldWith::new(self, init, fold_op) } /// Sums up the items in the iterator. @@ -1352,9 +1352,8 @@ pub trait ParallelIterator: Sized + Send { K: Ord + Send, F: Sync + Send + Fn(&Self::Item) -> K, { - self.map(|x| (f(&x), x)) - .min_by(|a, b| (a.0).cmp(&b.0)) - .map(|(_, x)| x) + let (_, x) = self.map(|x| (f(&x), x)).min_by(|a, b| (a.0).cmp(&b.0))?; + Some(x) } /// Computes the maximum of all the items in the iterator. If the @@ -1436,9 +1435,8 @@ pub trait ParallelIterator: Sized + Send { K: Ord + Send, F: Sync + Send + Fn(&Self::Item) -> K, { - self.map(|x| (f(&x), x)) - .max_by(|a, b| (a.0).cmp(&b.0)) - .map(|(_, x)| x) + let (_, x) = self.map(|x| (f(&x), x)).max_by(|a, b| (a.0).cmp(&b.0))?; + Some(x) } /// Takes two iterators and creates a new iterator over both. @@ -1461,7 +1459,7 @@ pub trait ParallelIterator: Sized + Send { where C: IntoParallelIterator, { - chain::new(self, chain.into_par_iter()) + Chain::new(self, chain.into_par_iter()) } /// Searches for **some** item in the parallel iterator that @@ -1729,7 +1727,7 @@ pub trait ParallelIterator: Sized + Send { Self: ParallelIterator>, T: Send, { - while_some::new(self) + WhileSome::new(self) } /// Wraps an iterator with a fuse in case of panics, to halt all threads @@ -1763,7 +1761,7 @@ pub trait ParallelIterator: Sized + Send { /// }); /// ``` fn panic_fuse(self) -> PanicFuse { - panic_fuse::new(self) + PanicFuse::new(self) } /// Create a fresh collection containing all the element produced @@ -1935,7 +1933,7 @@ pub trait ParallelIterator: Sized + Send { where Self::Item: Clone, { - intersperse::new(self, element) + Intersperse::new(self, element) } /// Internal method used to define the behavior of this parallel @@ -2063,7 +2061,7 @@ pub trait IndexedParallelIterator: ParallelIterator { Z: IntoParallelIterator, Z::Iter: IndexedParallelIterator, { - zip::new(self, zip_op.into_par_iter()) + Zip::new(self, zip_op.into_par_iter()) } /// The same as `Zip`, but requires that both iterators have the same length. @@ -2092,7 +2090,7 @@ pub trait IndexedParallelIterator: ParallelIterator { { let zip_op_iter = zip_op.into_par_iter(); assert_eq!(self.len(), zip_op_iter.len()); - zip_eq::new(self, zip_op_iter) + ZipEq::new(self, zip_op_iter) } /// Interleave elements of this iterator and the other given @@ -2114,7 +2112,7 @@ pub trait IndexedParallelIterator: ParallelIterator { I: IntoParallelIterator, I::Iter: IndexedParallelIterator, { - interleave::new(self, other.into_par_iter()) + Interleave::new(self, other.into_par_iter()) } /// Interleave elements of this iterator and the other given @@ -2133,7 +2131,7 @@ pub trait IndexedParallelIterator: ParallelIterator { I: IntoParallelIterator, I::Iter: IndexedParallelIterator, { - interleave_shortest::new(self, other.into_par_iter()) + InterleaveShortest::new(self, other.into_par_iter()) } /// Split an iterator up into fixed-size chunks. @@ -2158,7 +2156,7 @@ pub trait IndexedParallelIterator: ParallelIterator { /// ``` fn chunks(self, chunk_size: usize) -> Chunks { assert!(chunk_size != 0, "chunk_size must not be zero"); - chunks::new(self, chunk_size) + Chunks::new(self, chunk_size) } /// Lexicographically compares the elements of this `ParallelIterator` with those of @@ -2212,11 +2210,11 @@ pub trait IndexedParallelIterator: ParallelIterator { Self::Item: PartialOrd, { let other = other.into_par_iter(); - let ord_len = self.len().cmp(&other.len()); + let ord_len = Some(self.len().cmp(&other.len())); self.zip(other) .map(|(x, y)| PartialOrd::partial_cmp(&x, &y)) .find_first(|&ord| ord != Some(Ordering::Equal)) - .unwrap_or(Some(ord_len)) + .unwrap_or(ord_len) } /// Determines if the elements of this `ParallelIterator` @@ -2304,7 +2302,7 @@ pub trait IndexedParallelIterator: ParallelIterator { /// assert_eq!(result, [(0, 'a'), (1, 'b'), (2, 'c')]); /// ``` fn enumerate(self) -> Enumerate { - enumerate::new(self) + Enumerate::new(self) } /// Creates an iterator that skips the first `n` elements. @@ -2322,7 +2320,7 @@ pub trait IndexedParallelIterator: ParallelIterator { /// assert_eq!(result, [95, 96, 97, 98, 99]); /// ``` fn skip(self, n: usize) -> Skip { - skip::new(self, n) + Skip::new(self, n) } /// Creates an iterator that yields the first `n` elements. @@ -2340,7 +2338,7 @@ pub trait IndexedParallelIterator: ParallelIterator { /// assert_eq!(result, [0, 1, 2, 3, 4]); /// ``` fn take(self, n: usize) -> Take { - take::new(self, n) + Take::new(self, n) } /// Searches for **some** item in the parallel iterator that @@ -2365,10 +2363,8 @@ pub trait IndexedParallelIterator: ParallelIterator { where P: Fn(Self::Item) -> bool + Sync + Send, { - self.map(predicate) - .enumerate() - .find_any(|&(_, p)| p) - .map(|(i, _)| i) + let (i, _) = self.map(predicate).enumerate().find_any(|&(_, p)| p)?; + Some(i) } /// Searches for the sequentially **first** item in the parallel iterator @@ -2399,10 +2395,8 @@ pub trait IndexedParallelIterator: ParallelIterator { where P: Fn(Self::Item) -> bool + Sync + Send, { - self.map(predicate) - .enumerate() - .find_first(|&(_, p)| p) - .map(|(i, _)| i) + let (i, _) = self.map(predicate).enumerate().find_first(|&(_, p)| p)?; + Some(i) } /// Searches for the sequentially **last** item in the parallel iterator @@ -2433,10 +2427,8 @@ pub trait IndexedParallelIterator: ParallelIterator { where P: Fn(Self::Item) -> bool + Sync + Send, { - self.map(predicate) - .enumerate() - .find_last(|&(_, p)| p) - .map(|(i, _)| i) + let (i, _) = self.map(predicate).enumerate().find_last(|&(_, p)| p)?; + Some(i) } #[doc(hidden)] @@ -2467,7 +2459,7 @@ pub trait IndexedParallelIterator: ParallelIterator { /// assert_eq!(result, [4, 3, 2, 1, 0]); /// ``` fn rev(self) -> Rev { - rev::new(self) + Rev::new(self) } /// Sets the minimum length of iterators desired to process in each @@ -2493,7 +2485,7 @@ pub trait IndexedParallelIterator: ParallelIterator { /// assert!(min >= 1234); /// ``` fn with_min_len(self, min: usize) -> MinLen { - len::new_min_len(self, min) + MinLen::new(self, min) } /// Sets the maximum length of iterators desired to process in each @@ -2521,7 +2513,7 @@ pub trait IndexedParallelIterator: ParallelIterator { /// assert!(max <= 1234); /// ``` fn with_max_len(self, max: usize) -> MaxLen { - len::new_max_len(self, max) + MaxLen::new(self, max) } /// Produces an exact count of how many items this iterator will diff --git a/src/iter/noop.rs b/src/iter/noop.rs index 9e8bbb5b7..31d7528ed 100644 --- a/src/iter/noop.rs +++ b/src/iter/noop.rs @@ -1,9 +1,9 @@ use super::plumbing::*; -pub struct NoopConsumer; +pub(super) struct NoopConsumer; impl NoopConsumer { - pub fn new() -> Self { + pub(super) fn new() -> Self { NoopConsumer } } @@ -58,7 +58,7 @@ impl UnindexedConsumer for NoopConsumer { } } -pub struct NoopReducer; +pub(super) struct NoopReducer; impl Reducer<()> for NoopReducer { fn reduce(self, _left: (), _right: ()) {} diff --git a/src/iter/once.rs b/src/iter/once.rs index 8e0dc41f6..3819c20e8 100644 --- a/src/iter/once.rs +++ b/src/iter/once.rs @@ -22,7 +22,7 @@ use iter::*; /// assert_eq!(pi.position_any(|x| x < 0), Some(1234)); /// ``` pub fn once(item: T) -> Once { - Once { item: item } + Once { item } } /// Iterator adaptor for [the `once()` function](fn.once.html). diff --git a/src/iter/panic_fuse.rs b/src/iter/panic_fuse.rs index 9476cf439..d0cbbd4c1 100644 --- a/src/iter/panic_fuse.rs +++ b/src/iter/panic_fuse.rs @@ -36,14 +36,14 @@ impl<'a> Fuse<'a> { } } -/// Create a new `PanicFuse` iterator. -/// -/// NB: a free fn because it is NOT part of the end-user API. -pub fn new(base: I) -> PanicFuse +impl PanicFuse where I: ParallelIterator, { - PanicFuse { base: base } + /// Create a new `PanicFuse` iterator. + pub(super) fn new(base: I) -> PanicFuse { + PanicFuse { base } + } } impl ParallelIterator for PanicFuse @@ -93,7 +93,7 @@ where where CB: ProducerCallback, { - return self.base.with_producer(Callback { callback: callback }); + return self.base.with_producer(Callback { callback }); struct Callback { callback: CB, @@ -111,7 +111,7 @@ where { let panicked = AtomicBool::new(false); let producer = PanicFuseProducer { - base: base, + base, fuse: Fuse(&panicked), }; self.callback.callback(producer) diff --git a/src/iter/par_bridge.rs b/src/iter/par_bridge.rs index b4f2155c5..b366de698 100644 --- a/src/iter/par_bridge.rs +++ b/src/iter/par_bridge.rs @@ -1,4 +1,4 @@ -use crossbeam_deque::{Deque, Steal, Stealer}; +use crossbeam_deque::{self as deque, Steal, Stealer, Worker}; use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering}; use std::sync::{Mutex, TryLockError}; @@ -79,10 +79,9 @@ where C: UnindexedConsumer, { let split_count = AtomicUsize::new(current_num_threads()); - let deque = Deque::new(); - let stealer = deque.stealer(); + let (worker, stealer) = deque::fifo(); let done = AtomicBool::new(false); - let iter = Mutex::new((self.iter, deque)); + let iter = Mutex::new((self.iter, worker)); bridge_unindexed( IterParallelProducer { @@ -99,7 +98,7 @@ where struct IterParallelProducer<'a, Iter: Iterator + 'a> { split_count: &'a AtomicUsize, done: &'a AtomicBool, - iter: &'a Mutex<(Iter, Deque)>, + iter: &'a Mutex<(Iter, Worker)>, items: Stealer, } @@ -167,11 +166,15 @@ where let count = current_num_threads(); let count = (count * count) * 2; - let (ref mut iter, ref deque) = *guard; + let (ref mut iter, ref worker) = *guard; - while deque.len() < count { + // while worker.len() < count { + // FIXME the new deque doesn't let us count items. We can just + // push a number of items, but that doesn't consider active + // stealers elsewhere. + for _ in 0..count { if let Some(it) = iter.next() { - deque.push(it); + worker.push(it); } else { self.done.store(true, Ordering::SeqCst); break; diff --git a/src/iter/plumbing/mod.rs b/src/iter/plumbing/mod.rs index 6056ff974..d6cc5a169 100644 --- a/src/iter/plumbing/mod.rs +++ b/src/iter/plumbing/mod.rs @@ -269,7 +269,7 @@ impl Splitter { } #[inline] - fn try(&mut self, stolen: bool) -> bool { + fn try_split(&mut self, stolen: bool) -> bool { let Splitter { splits } = *self; if stolen { @@ -331,9 +331,9 @@ impl LengthSplitter { } #[inline] - fn try(&mut self, len: usize, stolen: bool) -> bool { + fn try_split(&mut self, len: usize, stolen: bool) -> bool { // If splitting wouldn't make us too small, try the inner splitter. - len / 2 >= self.min && self.inner.try(stolen) + len / 2 >= self.min && self.inner.try_split(stolen) } } @@ -354,10 +354,7 @@ where C: Consumer, { let len = par_iter.len(); - return par_iter.with_producer(Callback { - len: len, - consumer: consumer, - }); + return par_iter.with_producer(Callback { len, consumer }); struct Callback { len: usize, @@ -412,7 +409,7 @@ where { if consumer.full() { consumer.into_folder().complete() - } else if splitter.try(len, migrated) { + } else if splitter.try_split(len, migrated) { let mid = len / 2; let (left_producer, right_producer) = producer.split_at(mid); let (left_consumer, right_consumer, reducer) = consumer.split_at(mid); @@ -467,7 +464,7 @@ where { if consumer.full() { consumer.into_folder().complete() - } else if splitter.try(migrated) { + } else if splitter.try_split(migrated) { match producer.split() { (left_producer, Some(right_producer)) => { let (reducer, left_consumer, right_consumer) = diff --git a/src/iter/product.rs b/src/iter/product.rs index 2ee34263c..a3d0727f9 100644 --- a/src/iter/product.rs +++ b/src/iter/product.rs @@ -4,7 +4,7 @@ use super::ParallelIterator; use std::iter::{self, Product}; use std::marker::PhantomData; -pub fn product(pi: PI) -> P +pub(super) fn product(pi: PI) -> P where PI: ParallelIterator, P: Send + Product + Product, diff --git a/src/iter/reduce.rs b/src/iter/reduce.rs index ee3c909c9..8ba4f1a4e 100644 --- a/src/iter/reduce.rs +++ b/src/iter/reduce.rs @@ -1,7 +1,7 @@ use super::plumbing::*; use super::ParallelIterator; -pub fn reduce(pi: PI, identity: ID, reduce_op: R) -> T +pub(super) fn reduce(pi: PI, identity: ID, reduce_op: R) -> T where PI: ParallelIterator, R: Fn(T, T) -> T + Sync, diff --git a/src/iter/rev.rs b/src/iter/rev.rs index e617886bb..580a39485 100644 --- a/src/iter/rev.rs +++ b/src/iter/rev.rs @@ -13,14 +13,14 @@ pub struct Rev { base: I, } -/// Create a new `Rev` iterator. -/// -/// NB: a free fn because it is NOT part of the end-user API. -pub fn new(base: I) -> Rev +impl Rev where I: IndexedParallelIterator, { - Rev { base: base } + /// Create a new `Rev` iterator. + pub(super) fn new(base: I) -> Self { + Rev { base } + } } impl ParallelIterator for Rev @@ -58,10 +58,7 @@ where CB: ProducerCallback, { let len = self.base.len(); - return self.base.with_producer(Callback { - callback: callback, - len: len, - }); + return self.base.with_producer(Callback { callback, len }); struct Callback { callback: CB, @@ -78,7 +75,7 @@ where P: Producer, { let producer = RevProducer { - base: base, + base, len: self.len, }; self.callback.callback(producer) diff --git a/src/iter/skip.rs b/src/iter/skip.rs index f129e3b91..73653f24e 100644 --- a/src/iter/skip.rs +++ b/src/iter/skip.rs @@ -15,15 +15,15 @@ pub struct Skip { n: usize, } -/// Create a new `Skip` iterator. -/// -/// NB: a free fn because it is NOT part of the end-user API. -pub fn new(base: I, n: usize) -> Skip +impl Skip where I: IndexedParallelIterator, { - let n = min(base.len(), n); - Skip { base: base, n: n } + /// Create a new `Skip` iterator. + pub(super) fn new(base: I, n: usize) -> Self { + let n = min(base.len(), n); + Skip { base, n } + } } impl ParallelIterator for Skip @@ -61,7 +61,7 @@ where CB: ProducerCallback, { return self.base.with_producer(Callback { - callback: callback, + callback, n: self.n, }); diff --git a/src/iter/splitter.rs b/src/iter/splitter.rs index a952dcff4..45b0081c1 100644 --- a/src/iter/splitter.rs +++ b/src/iter/splitter.rs @@ -108,10 +108,7 @@ where D: Send, S: Fn(D) -> (D, Option) + Sync, { - Split { - data: data, - splitter: splitter, - } + Split { data, splitter } } /// `Split` is a parallel iterator using arbitrary data and a splitting function. @@ -125,7 +122,7 @@ pub struct Split { } impl Debug for Split { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("Split").field("data", &self.data).finish() } } @@ -165,13 +162,7 @@ where let splitter = self.splitter; let (left, right) = splitter(self.data); self.data = left; - ( - self, - right.map(|data| SplitProducer { - data: data, - splitter: splitter, - }), - ) + (self, right.map(|data| SplitProducer { data, splitter })) } fn fold_with(self, folder: F) -> F diff --git a/src/iter/sum.rs b/src/iter/sum.rs index 541e57a4a..a73e0bf3d 100644 --- a/src/iter/sum.rs +++ b/src/iter/sum.rs @@ -4,7 +4,7 @@ use super::ParallelIterator; use std::iter::{self, Sum}; use std::marker::PhantomData; -pub fn sum(pi: PI) -> S +pub(super) fn sum(pi: PI) -> S where PI: ParallelIterator, S: Send + Sum + Sum, diff --git a/src/iter/take.rs b/src/iter/take.rs index dee12372d..6bd73b27e 100644 --- a/src/iter/take.rs +++ b/src/iter/take.rs @@ -14,15 +14,15 @@ pub struct Take { n: usize, } -/// Create a new `Take` iterator. -/// -/// NB: a free fn because it is NOT part of the end-user API. -pub fn new(base: I, n: usize) -> Take +impl Take where I: IndexedParallelIterator, { - let n = min(base.len(), n); - Take { base: base, n: n } + /// Create a new `Take` iterator. + pub(super) fn new(base: I, n: usize) -> Self { + let n = min(base.len(), n); + Take { base, n } + } } impl ParallelIterator for Take @@ -60,7 +60,7 @@ where CB: ProducerCallback, { return self.base.with_producer(Callback { - callback: callback, + callback, n: self.n, }); diff --git a/src/iter/test.rs b/src/iter/test.rs index e3953dbe9..e920986a5 100644 --- a/src/iter/test.rs +++ b/src/iter/test.rs @@ -24,7 +24,7 @@ fn seeded_rng() -> XorShiftRng { } #[test] -pub fn execute() { +fn execute() { let a: Vec = (0..1024).collect(); let mut b = vec![]; a.par_iter().map(|&i| i + 1).collect_into_vec(&mut b); @@ -33,7 +33,7 @@ pub fn execute() { } #[test] -pub fn execute_cloned() { +fn execute_cloned() { let a: Vec = (0..1024).collect(); let mut b: Vec = vec![]; a.par_iter().cloned().collect_into_vec(&mut b); @@ -42,7 +42,7 @@ pub fn execute_cloned() { } #[test] -pub fn execute_range() { +fn execute_range() { let a = 0i32..1024; let mut b = vec![]; a.into_par_iter().map(|i| i + 1).collect_into_vec(&mut b); @@ -51,16 +51,15 @@ pub fn execute_range() { } #[test] -pub fn execute_unindexed_range() { +fn execute_unindexed_range() { let a = 0i64..1024; let b: LinkedList = a.into_par_iter().map(|i| i + 1).collect(); let c: LinkedList = (0..1024).map(|i| i + 1).collect(); assert_eq!(b, c); } -#[cfg(has_i128)] #[test] -pub fn execute_pseudo_indexed_range() { +fn execute_pseudo_indexed_range() { use std::i128::MAX; let range = MAX - 1024..MAX; @@ -74,37 +73,37 @@ pub fn execute_pseudo_indexed_range() { } #[test] -pub fn check_map_indexed() { +fn check_map_indexed() { let a = [1, 2, 3]; is_indexed(a.par_iter().map(|x| x)); } #[test] -pub fn map_sum() { +fn map_sum() { let a: Vec = (0..1024).collect(); let r1: i32 = a.par_iter().map(|&i| i + 1).sum(); - let r2 = a.iter().map(|&i| i + 1).fold(0, |a, b| a + b); + let r2 = a.iter().map(|&i| i + 1).sum(); assert_eq!(r1, r2); } #[test] -pub fn map_reduce() { +fn map_reduce() { let a: Vec = (0..1024).collect(); let r1 = a.par_iter().map(|&i| i + 1).reduce(|| 0, |i, j| i + j); - let r2 = a.iter().map(|&i| i + 1).fold(0, |a, b| a + b); + let r2 = a.iter().map(|&i| i + 1).sum(); assert_eq!(r1, r2); } #[test] -pub fn map_reduce_with() { +fn map_reduce_with() { let a: Vec = (0..1024).collect(); let r1 = a.par_iter().map(|&i| i + 1).reduce_with(|i, j| i + j); - let r2 = a.iter().map(|&i| i + 1).fold(0, |a, b| a + b); + let r2 = a.iter().map(|&i| i + 1).sum(); assert_eq!(r1, Some(r2)); } #[test] -pub fn fold_map_reduce() { +fn fold_map_reduce() { // Kind of a weird test, but it demonstrates various // transformations that are taking place. Relies on // `with_max_len(1).fold()` being equivalent to `map()`. @@ -170,7 +169,7 @@ pub fn fold_map_reduce() { } #[test] -pub fn fold_is_full() { +fn fold_is_full() { let counter = AtomicUsize::new(0); let a = (0_i32..2048) .into_par_iter() @@ -184,7 +183,7 @@ pub fn fold_is_full() { } #[test] -pub fn check_enumerate() { +fn check_enumerate() { let a: Vec = (0..1024).rev().collect(); let mut b = vec![]; @@ -196,7 +195,7 @@ pub fn check_enumerate() { } #[test] -pub fn check_enumerate_rev() { +fn check_enumerate_rev() { let a: Vec = (0..1024).rev().collect(); let mut b = vec![]; @@ -209,7 +208,7 @@ pub fn check_enumerate_rev() { } #[test] -pub fn check_indices_after_enumerate_split() { +fn check_indices_after_enumerate_split() { let a: Vec = (0..1024).collect(); a.par_iter().enumerate().with_producer(WithProducer); @@ -234,7 +233,7 @@ pub fn check_indices_after_enumerate_split() { } #[test] -pub fn check_increment() { +fn check_increment() { let mut a: Vec = (0..1024).rev().collect(); a.par_iter_mut().enumerate().for_each(|(i, v)| *v += i); @@ -243,7 +242,7 @@ pub fn check_increment() { } #[test] -pub fn check_skip() { +fn check_skip() { let a: Vec = (0..1024).collect(); let mut v1 = Vec::new(); @@ -272,7 +271,7 @@ pub fn check_skip() { } #[test] -pub fn check_take() { +fn check_take() { let a: Vec = (0..1024).collect(); let mut v1 = Vec::new(); @@ -292,7 +291,7 @@ pub fn check_take() { } #[test] -pub fn check_inspect() { +fn check_inspect() { use std::sync::atomic::{AtomicUsize, Ordering}; let a = AtomicUsize::new(0); @@ -307,7 +306,7 @@ pub fn check_inspect() { } #[test] -pub fn check_move() { +fn check_move() { let a = vec![vec![1, 2, 3]]; let ptr = a[0].as_ptr(); @@ -319,7 +318,7 @@ pub fn check_move() { } #[test] -pub fn check_drops() { +fn check_drops() { use std::sync::atomic::{AtomicUsize, Ordering}; let c = AtomicUsize::new(0); @@ -357,30 +356,30 @@ pub fn check_drops() { } #[test] -pub fn check_slice_indexed() { +fn check_slice_indexed() { let a = vec![1, 2, 3]; is_indexed(a.par_iter()); } #[test] -pub fn check_slice_mut_indexed() { +fn check_slice_mut_indexed() { let mut a = vec![1, 2, 3]; is_indexed(a.par_iter_mut()); } #[test] -pub fn check_vec_indexed() { +fn check_vec_indexed() { let a = vec![1, 2, 3]; is_indexed(a.clone().into_par_iter()); } #[test] -pub fn check_range_indexed() { +fn check_range_indexed() { is_indexed((1..5).into_par_iter()); } #[test] -pub fn check_cmp_direct() { +fn check_cmp_direct() { let a = (0..1024).into_par_iter(); let b = (0..1024).into_par_iter(); @@ -390,7 +389,7 @@ pub fn check_cmp_direct() { } #[test] -pub fn check_cmp_to_seq() { +fn check_cmp_to_seq() { assert_eq!( (0..1024).into_par_iter().cmp(0..1024), (0..1024).cmp(0..1024) @@ -398,7 +397,7 @@ pub fn check_cmp_to_seq() { } #[test] -pub fn check_cmp_rng_to_seq() { +fn check_cmp_rng_to_seq() { let mut rng = seeded_rng(); let a: Vec = rng.sample_iter(&Standard).take(1024).collect(); let b: Vec = rng.sample_iter(&Standard).take(1024).collect(); @@ -411,7 +410,7 @@ pub fn check_cmp_rng_to_seq() { } #[test] -pub fn check_cmp_lt_direct() { +fn check_cmp_lt_direct() { let a = (0..1024).into_par_iter(); let b = (1..1024).into_par_iter(); @@ -421,7 +420,7 @@ pub fn check_cmp_lt_direct() { } #[test] -pub fn check_cmp_lt_to_seq() { +fn check_cmp_lt_to_seq() { assert_eq!( (0..1024).into_par_iter().cmp(1..1024), (0..1024).cmp(1..1024) @@ -429,7 +428,7 @@ pub fn check_cmp_lt_to_seq() { } #[test] -pub fn check_cmp_gt_direct() { +fn check_cmp_gt_direct() { let a = (1..1024).into_par_iter(); let b = (0..1024).into_par_iter(); @@ -439,7 +438,7 @@ pub fn check_cmp_gt_direct() { } #[test] -pub fn check_cmp_gt_to_seq() { +fn check_cmp_gt_to_seq() { assert_eq!( (1..1024).into_par_iter().cmp(0..1024), (1..1024).cmp(0..1024) @@ -447,7 +446,7 @@ pub fn check_cmp_gt_to_seq() { } #[test] -pub fn check_cmp_short_circuit() { +fn check_cmp_short_circuit() { let a = vec![0; 1024]; let mut b = a.clone(); b[42] = 1; @@ -464,7 +463,7 @@ pub fn check_cmp_short_circuit() { } #[test] -pub fn check_partial_cmp_short_circuit() { +fn check_partial_cmp_short_circuit() { let a = vec![0; 1024]; let mut b = a.clone(); b[42] = 1; @@ -481,7 +480,7 @@ pub fn check_partial_cmp_short_circuit() { } #[test] -pub fn check_partial_cmp_nan_short_circuit() { +fn check_partial_cmp_nan_short_circuit() { let a = vec![0.0; 1024]; let mut b = a.clone(); b[42] = f64::NAN; @@ -498,7 +497,7 @@ pub fn check_partial_cmp_nan_short_circuit() { } #[test] -pub fn check_partial_cmp_direct() { +fn check_partial_cmp_direct() { let a = (0..1024).into_par_iter(); let b = (0..1024).into_par_iter(); @@ -508,14 +507,14 @@ pub fn check_partial_cmp_direct() { } #[test] -pub fn check_partial_cmp_to_seq() { +fn check_partial_cmp_to_seq() { let par_result = (0..1024).into_par_iter().partial_cmp(0..1024); let seq_result = (0..1024).partial_cmp(0..1024); assert_eq!(par_result, seq_result); } #[test] -pub fn check_partial_cmp_rng_to_seq() { +fn check_partial_cmp_rng_to_seq() { let mut rng = seeded_rng(); let a: Vec = rng.sample_iter(&Standard).take(1024).collect(); let b: Vec = rng.sample_iter(&Standard).take(1024).collect(); @@ -528,7 +527,7 @@ pub fn check_partial_cmp_rng_to_seq() { } #[test] -pub fn check_partial_cmp_lt_direct() { +fn check_partial_cmp_lt_direct() { let a = (0..1024).into_par_iter(); let b = (1..1024).into_par_iter(); @@ -538,14 +537,14 @@ pub fn check_partial_cmp_lt_direct() { } #[test] -pub fn check_partial_cmp_lt_to_seq() { +fn check_partial_cmp_lt_to_seq() { let par_result = (0..1024).into_par_iter().partial_cmp(1..1024); let seq_result = (0..1024).partial_cmp(1..1024); assert_eq!(par_result, seq_result); } #[test] -pub fn check_partial_cmp_gt_direct() { +fn check_partial_cmp_gt_direct() { let a = (1..1024).into_par_iter(); let b = (0..1024).into_par_iter(); @@ -555,14 +554,14 @@ pub fn check_partial_cmp_gt_direct() { } #[test] -pub fn check_partial_cmp_gt_to_seq() { +fn check_partial_cmp_gt_to_seq() { let par_result = (1..1024).into_par_iter().partial_cmp(0..1024); let seq_result = (1..1024).partial_cmp(0..1024); assert_eq!(par_result, seq_result); } #[test] -pub fn check_partial_cmp_none_direct() { +fn check_partial_cmp_none_direct() { let a = vec![f64::NAN, 0.0]; let b = vec![0.0, 1.0]; @@ -572,7 +571,7 @@ pub fn check_partial_cmp_none_direct() { } #[test] -pub fn check_partial_cmp_none_to_seq() { +fn check_partial_cmp_none_to_seq() { let a = vec![f64::NAN, 0.0]; let b = vec![0.0, 1.0]; @@ -583,7 +582,7 @@ pub fn check_partial_cmp_none_to_seq() { } #[test] -pub fn check_partial_cmp_late_nan_direct() { +fn check_partial_cmp_late_nan_direct() { let a = vec![0.0, f64::NAN]; let b = vec![1.0, 1.0]; @@ -593,7 +592,7 @@ pub fn check_partial_cmp_late_nan_direct() { } #[test] -pub fn check_partial_cmp_late_nane_to_seq() { +fn check_partial_cmp_late_nane_to_seq() { let a = vec![0.0, f64::NAN]; let b = vec![1.0, 1.0]; @@ -604,7 +603,7 @@ pub fn check_partial_cmp_late_nane_to_seq() { } #[test] -pub fn check_cmp_lengths() { +fn check_cmp_lengths() { // comparisons should consider length if they are otherwise equal let a = vec![0; 1024]; let b = vec![0; 1025]; @@ -614,7 +613,7 @@ pub fn check_cmp_lengths() { } #[test] -pub fn check_eq_direct() { +fn check_eq_direct() { let a = (0..1024).into_par_iter(); let b = (0..1024).into_par_iter(); @@ -624,7 +623,7 @@ pub fn check_eq_direct() { } #[test] -pub fn check_eq_to_seq() { +fn check_eq_to_seq() { let par_result = (0..1024).into_par_iter().eq((0..1024).into_par_iter()); let seq_result = (0..1024).eq(0..1024); @@ -632,7 +631,7 @@ pub fn check_eq_to_seq() { } #[test] -pub fn check_ne_direct() { +fn check_ne_direct() { let a = (0..1024).into_par_iter(); let b = (1..1024).into_par_iter(); @@ -642,7 +641,7 @@ pub fn check_ne_direct() { } #[test] -pub fn check_ne_to_seq() { +fn check_ne_to_seq() { let par_result = (0..1024).into_par_iter().ne((1..1025).into_par_iter()); let seq_result = (0..1024).ne(1..1025); @@ -650,7 +649,7 @@ pub fn check_ne_to_seq() { } #[test] -pub fn check_ne_lengths() { +fn check_ne_lengths() { // equality should consider length too let a = vec![0; 1024]; let b = vec![0; 1025]; @@ -660,13 +659,13 @@ pub fn check_ne_lengths() { } #[test] -pub fn check_lt_direct() { +fn check_lt_direct() { assert!((0..1024).into_par_iter().lt(1..1024)); assert!(!(1..1024).into_par_iter().lt(0..1024)); } #[test] -pub fn check_lt_to_seq() { +fn check_lt_to_seq() { let par_result = (0..1024).into_par_iter().lt((1..1024).into_par_iter()); let seq_result = (0..1024).lt(1..1024); @@ -674,12 +673,12 @@ pub fn check_lt_to_seq() { } #[test] -pub fn check_le_equal_direct() { +fn check_le_equal_direct() { assert!((0..1024).into_par_iter().le((0..1024).into_par_iter())); } #[test] -pub fn check_le_equal_to_seq() { +fn check_le_equal_to_seq() { let par_result = (0..1024).into_par_iter().le((0..1024).into_par_iter()); let seq_result = (0..1024).le(0..1024); @@ -687,12 +686,12 @@ pub fn check_le_equal_to_seq() { } #[test] -pub fn check_le_less_direct() { +fn check_le_less_direct() { assert!((0..1024).into_par_iter().le((1..1024).into_par_iter())); } #[test] -pub fn check_le_less_to_seq() { +fn check_le_less_to_seq() { let par_result = (0..1024).into_par_iter().le((1..1024).into_par_iter()); let seq_result = (0..1024).le(1..1024); @@ -700,12 +699,12 @@ pub fn check_le_less_to_seq() { } #[test] -pub fn check_gt_direct() { +fn check_gt_direct() { assert!((1..1024).into_par_iter().gt((0..1024).into_par_iter())); } #[test] -pub fn check_gt_to_seq() { +fn check_gt_to_seq() { let par_result = (1..1024).into_par_iter().gt((0..1024).into_par_iter()); let seq_result = (1..1024).gt(0..1024); @@ -713,12 +712,12 @@ pub fn check_gt_to_seq() { } #[test] -pub fn check_ge_equal_direct() { +fn check_ge_equal_direct() { assert!((0..1024).into_par_iter().ge((0..1024).into_par_iter())); } #[test] -pub fn check_ge_equal_to_seq() { +fn check_ge_equal_to_seq() { let par_result = (0..1024).into_par_iter().ge((0..1024).into_par_iter()); let seq_result = (0..1024).ge(0..1024); @@ -726,12 +725,12 @@ pub fn check_ge_equal_to_seq() { } #[test] -pub fn check_ge_greater_direct() { +fn check_ge_greater_direct() { assert!((1..1024).into_par_iter().ge((0..1024).into_par_iter())); } #[test] -pub fn check_ge_greater_to_seq() { +fn check_ge_greater_to_seq() { let par_result = (1..1024).into_par_iter().ge((0..1024).into_par_iter()); let seq_result = (1..1024).ge(0..1024); @@ -739,7 +738,7 @@ pub fn check_ge_greater_to_seq() { } #[test] -pub fn check_zip() { +fn check_zip() { let mut a: Vec = (0..1024).rev().collect(); let b: Vec = (0..1024).collect(); @@ -749,7 +748,7 @@ pub fn check_zip() { } #[test] -pub fn check_zip_into_par_iter() { +fn check_zip_into_par_iter() { let mut a: Vec = (0..1024).rev().collect(); let b: Vec = (0..1024).collect(); @@ -761,7 +760,7 @@ pub fn check_zip_into_par_iter() { } #[test] -pub fn check_zip_into_mut_par_iter() { +fn check_zip_into_mut_par_iter() { let a: Vec = (0..1024).rev().collect(); let mut b: Vec = (0..1024).collect(); @@ -771,7 +770,7 @@ pub fn check_zip_into_mut_par_iter() { } #[test] -pub fn check_zip_range() { +fn check_zip_range() { let mut a: Vec = (0..1024).rev().collect(); a.par_iter_mut() @@ -782,7 +781,7 @@ pub fn check_zip_range() { } #[test] -pub fn check_zip_eq() { +fn check_zip_eq() { let mut a: Vec = (0..1024).rev().collect(); let b: Vec = (0..1024).collect(); @@ -792,7 +791,7 @@ pub fn check_zip_eq() { } #[test] -pub fn check_zip_eq_into_par_iter() { +fn check_zip_eq_into_par_iter() { let mut a: Vec = (0..1024).rev().collect(); let b: Vec = (0..1024).collect(); @@ -804,7 +803,7 @@ pub fn check_zip_eq_into_par_iter() { } #[test] -pub fn check_zip_eq_into_mut_par_iter() { +fn check_zip_eq_into_mut_par_iter() { let a: Vec = (0..1024).rev().collect(); let mut b: Vec = (0..1024).collect(); @@ -814,7 +813,7 @@ pub fn check_zip_eq_into_mut_par_iter() { } #[test] -pub fn check_zip_eq_range() { +fn check_zip_eq_range() { let mut a: Vec = (0..1024).rev().collect(); a.par_iter_mut() @@ -825,33 +824,29 @@ pub fn check_zip_eq_range() { } #[test] -pub fn check_sum_filtered_ints() { +fn check_sum_filtered_ints() { let a: Vec = vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10]; let par_sum_evens: i32 = a.par_iter().filter(|&x| (x & 1) == 0).sum(); - let seq_sum_evens = a - .iter() - .filter(|&x| (x & 1) == 0) - .map(|&x| x) - .fold(0, |a, b| a + b); + let seq_sum_evens = a.iter().filter(|&x| (x & 1) == 0).sum(); assert_eq!(par_sum_evens, seq_sum_evens); } #[test] -pub fn check_sum_filtermap_ints() { +fn check_sum_filtermap_ints() { let a: Vec = vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10]; - let par_sum_evens: f32 = a + let par_sum_evens: u32 = a .par_iter() - .filter_map(|&x| if (x & 1) == 0 { Some(x as f32) } else { None }) + .filter_map(|&x| if (x & 1) == 0 { Some(x as u32) } else { None }) .sum(); let seq_sum_evens = a .iter() - .filter_map(|&x| if (x & 1) == 0 { Some(x as f32) } else { None }) + .filter_map(|&x| if (x & 1) == 0 { Some(x as u32) } else { None }) .sum(); assert_eq!(par_sum_evens, seq_sum_evens); } #[test] -pub fn check_flat_map_nested_ranges() { +fn check_flat_map_nested_ranges() { // FIXME -- why are precise type hints required on the integers here? let v: i32 = (0_i32..10) @@ -863,13 +858,13 @@ pub fn check_flat_map_nested_ranges() { let w = (0_i32..10) .flat_map(|i| (0_i32..10).map(move |j| (i, j))) .map(|(i, j)| i * j) - .fold(0, |i, j| i + j); + .sum(); assert_eq!(v, w); } #[test] -pub fn check_empty_flat_map_sum() { +fn check_empty_flat_map_sum() { let a: Vec = (0..1024).collect(); let empty = &a[..0]; @@ -883,7 +878,7 @@ pub fn check_empty_flat_map_sum() { } #[test] -pub fn check_flatten_vec() { +fn check_flatten_vec() { let a: Vec = (0..1024).collect(); let b: Vec> = vec![a.clone(), a.clone(), a.clone(), a.clone()]; let c: Vec = b.par_iter().flatten().cloned().collect(); @@ -896,7 +891,7 @@ pub fn check_flatten_vec() { } #[test] -pub fn check_flatten_vec_empty() { +fn check_flatten_vec_empty() { let a: Vec> = vec![vec![]]; let b: Vec = a.par_iter().flatten().cloned().collect(); @@ -904,7 +899,7 @@ pub fn check_flatten_vec_empty() { } #[test] -pub fn check_slice_split() { +fn check_slice_split() { let v: Vec<_> = (0..1000).collect(); for m in 1..100 { let a: Vec<_> = v.split(|x| x % m == 0).collect(); @@ -927,7 +922,7 @@ pub fn check_slice_split() { } #[test] -pub fn check_slice_split_mut() { +fn check_slice_split_mut() { let mut v1: Vec<_> = (0..1000).collect(); let mut v2 = v1.clone(); for m in 1..100 { @@ -945,56 +940,38 @@ pub fn check_slice_split_mut() { } #[test] -pub fn check_chunks() { +fn check_chunks() { let a: Vec = vec![1, 5, 10, 4, 100, 3, 1000, 2, 10000, 1]; - let par_sum_product_pairs: i32 = a - .par_chunks(2) - .map(|c| c.iter().map(|&x| x).fold(1, |i, j| i * j)) - .sum(); - let seq_sum_product_pairs = a - .chunks(2) - .map(|c| c.iter().map(|&x| x).fold(1, |i, j| i * j)) - .fold(0, |i, j| i + j); + let par_sum_product_pairs: i32 = a.par_chunks(2).map(|c| c.iter().product::()).sum(); + let seq_sum_product_pairs = a.chunks(2).map(|c| c.iter().product::()).sum(); assert_eq!(par_sum_product_pairs, 12345); assert_eq!(par_sum_product_pairs, seq_sum_product_pairs); - let par_sum_product_triples: i32 = a - .par_chunks(3) - .map(|c| c.iter().map(|&x| x).fold(1, |i, j| i * j)) - .sum(); - let seq_sum_product_triples = a - .chunks(3) - .map(|c| c.iter().map(|&x| x).fold(1, |i, j| i * j)) - .fold(0, |i, j| i + j); + let par_sum_product_triples: i32 = a.par_chunks(3).map(|c| c.iter().product::()).sum(); + let seq_sum_product_triples = a.chunks(3).map(|c| c.iter().product::()).sum(); assert_eq!(par_sum_product_triples, 5_0 + 12_00 + 2_000_0000 + 1); assert_eq!(par_sum_product_triples, seq_sum_product_triples); } #[test] -pub fn check_chunks_mut() { +fn check_chunks_mut() { let mut a: Vec = vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10]; let mut b: Vec = a.clone(); - a.par_chunks_mut(2) - .for_each(|c| c[0] = c.iter().map(|&x| x).fold(0, |i, j| i + j)); - b.chunks_mut(2) - .map(|c| c[0] = c.iter().map(|&x| x).fold(0, |i, j| i + j)) - .count(); + a.par_chunks_mut(2).for_each(|c| c[0] = c.iter().sum()); + b.chunks_mut(2).for_each(|c| c[0] = c.iter().sum()); assert_eq!(a, &[3, 2, 7, 4, 11, 6, 15, 8, 19, 10]); assert_eq!(a, b); let mut a: Vec = vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10]; let mut b: Vec = a.clone(); - a.par_chunks_mut(3) - .for_each(|c| c[0] = c.iter().map(|&x| x).fold(0, |i, j| i + j)); - b.chunks_mut(3) - .map(|c| c[0] = c.iter().map(|&x| x).fold(0, |i, j| i + j)) - .count(); + a.par_chunks_mut(3).for_each(|c| c[0] = c.iter().sum()); + b.chunks_mut(3).for_each(|c| c[0] = c.iter().sum()); assert_eq!(a, &[6, 2, 3, 15, 5, 6, 24, 8, 9, 10]); assert_eq!(a, b); } #[test] -pub fn check_windows() { +fn check_windows() { let a: Vec = (0..1024).collect(); let par: Vec<_> = a.par_windows(2).collect(); let seq: Vec<_> = a.windows(2).collect(); @@ -1022,7 +999,7 @@ pub fn check_windows() { } #[test] -pub fn check_options() { +fn check_options() { let mut a = vec![None, Some(1), None, None, Some(2), Some(4)]; assert_eq!(7, a.par_iter().flat_map(|opt| opt).sum::()); @@ -1036,7 +1013,7 @@ pub fn check_options() { } #[test] -pub fn check_results() { +fn check_results() { let mut a = vec![Err(()), Ok(1i32), Err(()), Err(()), Ok(2), Ok(4)]; assert_eq!(7, a.par_iter().flat_map(|res| res).sum::()); @@ -1055,7 +1032,7 @@ pub fn check_results() { } #[test] -pub fn check_binary_heap() { +fn check_binary_heap() { use std::collections::BinaryHeap; let a: BinaryHeap = (0..10).collect(); @@ -1065,7 +1042,7 @@ pub fn check_binary_heap() { } #[test] -pub fn check_btree_map() { +fn check_btree_map() { use std::collections::BTreeMap; let mut a: BTreeMap = (0..10).map(|i| (i, -i)).collect(); @@ -1079,7 +1056,7 @@ pub fn check_btree_map() { } #[test] -pub fn check_btree_set() { +fn check_btree_set() { use std::collections::BTreeSet; let a: BTreeSet = (0..10).collect(); @@ -1089,7 +1066,7 @@ pub fn check_btree_set() { } #[test] -pub fn check_hash_map() { +fn check_hash_map() { use std::collections::HashMap; let mut a: HashMap = (0..10).map(|i| (i, -i)).collect(); @@ -1103,7 +1080,7 @@ pub fn check_hash_map() { } #[test] -pub fn check_hash_set() { +fn check_hash_set() { use std::collections::HashSet; let a: HashSet = (0..10).collect(); @@ -1113,7 +1090,7 @@ pub fn check_hash_set() { } #[test] -pub fn check_linked_list() { +fn check_linked_list() { use std::collections::LinkedList; let mut a: LinkedList = (0..10).collect(); @@ -1126,7 +1103,7 @@ pub fn check_linked_list() { } #[test] -pub fn check_vec_deque() { +fn check_vec_deque() { use std::collections::VecDeque; let mut a: VecDeque = (0..10).collect(); @@ -1143,7 +1120,7 @@ pub fn check_vec_deque() { } #[test] -pub fn check_chain() { +fn check_chain() { let mut res = vec![]; // stays indexed in the face of madness @@ -1199,7 +1176,7 @@ pub fn check_chain() { } #[test] -pub fn check_count() { +fn check_count() { let c0 = (0_u32..24 * 1024).filter(|i| i % 2 == 0).count(); let c1 = (0_u32..24 * 1024) .into_par_iter() @@ -1209,7 +1186,7 @@ pub fn check_count() { } #[test] -pub fn find_any() { +fn find_any() { let a: Vec = (0..1024).collect(); assert!(a.par_iter().find_any(|&&x| x % 42 == 41).is_some()); @@ -1234,7 +1211,7 @@ pub fn find_any() { } #[test] -pub fn find_first_or_last() { +fn find_first_or_last() { let a: Vec = (0..1024).collect(); assert_eq!(a.par_iter().find_first(|&&x| x % 42 == 41), Some(&41_i32)); @@ -1273,7 +1250,7 @@ pub fn find_first_or_last() { } #[test] -pub fn find_map_first_or_last_or_any() { +fn find_map_first_or_last_or_any() { let mut a: Vec = vec![]; assert!(a.par_iter().find_map_any(half_if_positive).is_none()); @@ -1317,7 +1294,7 @@ pub fn find_map_first_or_last_or_any() { } #[test] -pub fn check_find_not_present() { +fn check_find_not_present() { let counter = AtomicUsize::new(0); let value: Option = (0_i32..2048).into_par_iter().find_any(|&p| { counter.fetch_add(1, Ordering::SeqCst); @@ -1328,7 +1305,7 @@ pub fn check_find_not_present() { } #[test] -pub fn check_find_is_present() { +fn check_find_is_present() { let counter = AtomicUsize::new(0); let value: Option = (0_i32..2048).into_par_iter().find_any(|&p| { counter.fetch_add(1, Ordering::SeqCst); @@ -1340,7 +1317,7 @@ pub fn check_find_is_present() { } #[test] -pub fn check_while_some() { +fn check_while_some() { let value = (0_i32..2048).into_par_iter().map(Some).while_some().max(); assert_eq!(value, Some(2047)); @@ -1362,7 +1339,7 @@ pub fn check_while_some() { } #[test] -pub fn par_iter_collect_option() { +fn par_iter_collect_option() { let a: Option> = (0_i32..2048).map(Some).collect(); let b: Option> = (0_i32..2048).into_par_iter().map(Some).collect(); assert_eq!(a, b); @@ -1375,7 +1352,7 @@ pub fn par_iter_collect_option() { } #[test] -pub fn par_iter_collect_result() { +fn par_iter_collect_result() { let a: Result, ()> = (0_i32..2048).map(Ok).collect(); let b: Result, ()> = (0_i32..2048).into_par_iter().map(Ok).collect(); assert_eq!(a, b); @@ -1394,7 +1371,7 @@ pub fn par_iter_collect_result() { } #[test] -pub fn par_iter_collect() { +fn par_iter_collect() { let a: Vec = (0..1024).collect(); let b: Vec = a.par_iter().map(|&i| i + 1).collect(); let c: Vec = (0..1024).map(|i| i + 1).collect(); @@ -1402,7 +1379,7 @@ pub fn par_iter_collect() { } #[test] -pub fn par_iter_collect_vecdeque() { +fn par_iter_collect_vecdeque() { let a: Vec = (0..1024).collect(); let b: VecDeque = a.par_iter().cloned().collect(); let c: VecDeque = a.iter().cloned().collect(); @@ -1410,7 +1387,7 @@ pub fn par_iter_collect_vecdeque() { } #[test] -pub fn par_iter_collect_binaryheap() { +fn par_iter_collect_binaryheap() { let a: Vec = (0..1024).collect(); let mut b: BinaryHeap = a.par_iter().cloned().collect(); assert_eq!(b.peek(), Some(&1023)); @@ -1422,7 +1399,7 @@ pub fn par_iter_collect_binaryheap() { } #[test] -pub fn par_iter_collect_hashmap() { +fn par_iter_collect_hashmap() { let a: Vec = (0..1024).collect(); let b: HashMap = a.par_iter().map(|&i| (i, format!("{}", i))).collect(); assert_eq!(&b[&3], "3"); @@ -1430,14 +1407,14 @@ pub fn par_iter_collect_hashmap() { } #[test] -pub fn par_iter_collect_hashset() { +fn par_iter_collect_hashset() { let a: Vec = (0..1024).collect(); let b: HashSet = a.par_iter().cloned().collect(); assert_eq!(b.len(), 1024); } #[test] -pub fn par_iter_collect_btreemap() { +fn par_iter_collect_btreemap() { let a: Vec = (0..1024).collect(); let b: BTreeMap = a.par_iter().map(|&i| (i, format!("{}", i))).collect(); assert_eq!(&b[&3], "3"); @@ -1445,14 +1422,14 @@ pub fn par_iter_collect_btreemap() { } #[test] -pub fn par_iter_collect_btreeset() { +fn par_iter_collect_btreeset() { let a: Vec = (0..1024).collect(); let b: BTreeSet = a.par_iter().cloned().collect(); assert_eq!(b.len(), 1024); } #[test] -pub fn par_iter_collect_linked_list() { +fn par_iter_collect_linked_list() { let a: Vec = (0..1024).collect(); let b: LinkedList<_> = a.par_iter().map(|&i| (i, format!("{}", i))).collect(); let c: LinkedList<_> = a.iter().map(|&i| (i, format!("{}", i))).collect(); @@ -1460,7 +1437,7 @@ pub fn par_iter_collect_linked_list() { } #[test] -pub fn par_iter_collect_linked_list_flat_map_filter() { +fn par_iter_collect_linked_list_flat_map_filter() { let b: LinkedList = (0_i32..1024) .into_par_iter() .flat_map(|i| (0..i)) @@ -1474,39 +1451,36 @@ pub fn par_iter_collect_linked_list_flat_map_filter() { } #[test] -pub fn par_iter_collect_cows() { +fn par_iter_collect_cows() { use std::borrow::Cow; let s = "Fearless Concurrency with Rust"; // Collects `i32` into a `Vec` - let a: Cow<[i32]> = (0..1024).collect(); - let b: Cow<[i32]> = a.par_iter().cloned().collect(); + let a: Cow<'_, [i32]> = (0..1024).collect(); + let b: Cow<'_, [i32]> = a.par_iter().cloned().collect(); assert_eq!(a, b); // Collects `char` into a `String` - let a: Cow = s.chars().collect(); - let b: Cow = s.par_chars().collect(); + let a: Cow<'_, str> = s.chars().collect(); + let b: Cow<'_, str> = s.par_chars().collect(); assert_eq!(a, b); // Collects `str` into a `String` - let a: Cow = s.split_whitespace().collect(); - let b: Cow = s.par_split_whitespace().collect(); + let a: Cow<'_, str> = s.split_whitespace().collect(); + let b: Cow<'_, str> = s.par_split_whitespace().collect(); assert_eq!(a, b); // Collects `String` into a `String` - let a: Cow = s.split_whitespace().map(|s| s.to_owned()).collect(); - let b: Cow = s.par_split_whitespace().map(|s| s.to_owned()).collect(); + let a: Cow<'_, str> = s.split_whitespace().map(str::to_owned).collect(); + let b: Cow<'_, str> = s.par_split_whitespace().map(str::to_owned).collect(); assert_eq!(a, b); } #[test] -pub fn par_iter_unindexed_flat_map() { - let b: Vec = (0_i64..1024) - .into_par_iter() - .flat_map(|i| Some(i)) - .collect(); - let c: Vec = (0_i64..1024).flat_map(|i| Some(i)).collect(); +fn par_iter_unindexed_flat_map() { + let b: Vec = (0_i64..1024).into_par_iter().flat_map(Some).collect(); + let c: Vec = (0_i64..1024).flat_map(Some).collect(); assert_eq!(b, c); } @@ -1514,7 +1488,7 @@ pub fn par_iter_unindexed_flat_map() { fn min_max() { let mut rng = seeded_rng(); let a: Vec = rng.sample_iter(&Standard).take(1024).collect(); - for i in 0..a.len() + 1 { + for i in 0..=a.len() { let slice = &a[..i]; assert_eq!(slice.par_iter().min(), slice.iter().min()); assert_eq!(slice.par_iter().max(), slice.iter().max()); @@ -1527,7 +1501,7 @@ fn min_max_by() { // Make sure there are duplicate keys, for testing sort stability let r: Vec = rng.sample_iter(&Standard).take(512).collect(); let a: Vec<(i32, u16)> = r.iter().chain(&r).cloned().zip(0..).collect(); - for i in 0..a.len() + 1 { + for i in 0..=a.len() { let slice = &a[..i]; assert_eq!( slice.par_iter().min_by(|x, y| x.0.cmp(&y.0)), @@ -1546,7 +1520,7 @@ fn min_max_by_key() { // Make sure there are duplicate keys, for testing sort stability let r: Vec = rng.sample_iter(&Standard).take(512).collect(); let a: Vec<(i32, u16)> = r.iter().chain(&r).cloned().zip(0..).collect(); - for i in 0..a.len() + 1 { + for i in 0..=a.len() { let slice = &a[..i]; assert_eq!( slice.par_iter().min_by_key(|x| x.0), @@ -1577,7 +1551,7 @@ fn scope_mix() { s.spawn(move |_| { let a: Vec = (0..1024).collect(); let r1 = a.par_iter().map(|&i| i + 1).reduce_with(|i, j| i + j); - let r2 = a.iter().map(|&i| i + 1).fold(0, |a, b| a + b); + let r2 = a.iter().map(|&i| i + 1).sum(); assert_eq!(r1.unwrap(), r2); }); }); @@ -1634,7 +1608,7 @@ fn check_lengths() { ); } - let lengths = [0, 1, 10, 100, 1000, 10000, 100000, 1000000, usize::MAX]; + let lengths = [0, 1, 10, 100, 1_000, 10_000, 100_000, 1_000_000, usize::MAX]; for &min in &lengths { for &max in &lengths { check(min, max); @@ -1812,7 +1786,7 @@ fn check_unzip() { .into_par_iter() .filter_map(|i| Some((i, i * i))) .unzip(); - let (c, d): (Vec<_>, Vec<_>) = (0..1024).filter_map(|i| Some((i, i * i))).unzip(); + let (c, d): (Vec<_>, Vec<_>) = (0..1024).map(|i| (i, i * i)).unzip(); assert_eq!(a, c); assert_eq!(b, d); } diff --git a/src/iter/try_fold.rs b/src/iter/try_fold.rs index 2f3a9c011..13165c9ec 100644 --- a/src/iter/try_fold.rs +++ b/src/iter/try_fold.rs @@ -5,18 +5,20 @@ use super::private::Try; use std::fmt::{self, Debug}; use std::marker::PhantomData; -pub fn try_fold(base: I, identity: ID, fold_op: F) -> TryFold +impl TryFold where I: ParallelIterator, F: Fn(U::Ok, I::Item) -> U + Sync + Send, ID: Fn() -> U::Ok + Sync + Send, U: Try + Send, { - TryFold { - base: base, - identity: identity, - fold_op: fold_op, - marker: PhantomData, + pub(super) fn new(base: I, identity: ID, fold_op: F) -> Self { + TryFold { + base, + identity, + fold_op, + marker: PhantomData, + } } } @@ -35,7 +37,7 @@ pub struct TryFold { } impl Debug for TryFold { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("TryFold").field("base", &self.base).finish() } } @@ -142,10 +144,7 @@ where fn consume(self, item: T) -> Self { let fold_op = self.fold_op; let result = self.result.and_then(|acc| fold_op(acc, item).into_result()); - TryFoldFolder { - result: result, - ..self - } + TryFoldFolder { result, ..self } } fn complete(self) -> C::Result { @@ -163,17 +162,19 @@ where // /////////////////////////////////////////////////////////////////////////// -pub fn try_fold_with(base: I, item: U::Ok, fold_op: F) -> TryFoldWith +impl TryFoldWith where I: ParallelIterator, F: Fn(U::Ok, I::Item) -> U + Sync, U: Try + Send, U::Ok: Clone + Send, { - TryFoldWith { - base: base, - item: item, - fold_op: fold_op, + pub(super) fn new(base: I, item: U::Ok, fold_op: F) -> Self { + TryFoldWith { + base, + item, + fold_op, + } } } @@ -194,7 +195,7 @@ impl Debug for TryFoldWith where U::Ok: Debug, { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("TryFoldWith") .field("base", &self.base) .field("item", &self.item) diff --git a/src/iter/try_reduce.rs b/src/iter/try_reduce.rs index 765d15c23..926746456 100644 --- a/src/iter/try_reduce.rs +++ b/src/iter/try_reduce.rs @@ -4,7 +4,7 @@ use super::ParallelIterator; use super::private::Try; use std::sync::atomic::{AtomicBool, Ordering}; -pub fn try_reduce(pi: PI, identity: ID, reduce_op: R) -> T +pub(super) fn try_reduce(pi: PI, identity: ID, reduce_op: R) -> T where PI: ParallelIterator, R: Fn(T::Ok, T::Ok) -> T + Sync, @@ -110,10 +110,7 @@ where if result.is_err() { self.full.store(true, Ordering::Relaxed) } - TryReduceFolder { - result: result, - ..self - } + TryReduceFolder { result, ..self } } fn complete(self) -> T { diff --git a/src/iter/try_reduce_with.rs b/src/iter/try_reduce_with.rs index da84f12ee..137dd5886 100644 --- a/src/iter/try_reduce_with.rs +++ b/src/iter/try_reduce_with.rs @@ -4,7 +4,7 @@ use super::ParallelIterator; use super::private::Try; use std::sync::atomic::{AtomicBool, Ordering}; -pub fn try_reduce_with(pi: PI, reduce_op: R) -> Option +pub(super) fn try_reduce_with(pi: PI, reduce_op: R) -> Option where PI: ParallelIterator, R: Fn(T::Ok, T::Ok) -> T + Sync, @@ -121,7 +121,8 @@ where } fn complete(self) -> Option { - self.opt_result.map(|result| match result { + let result = self.opt_result?; + Some(match result { Ok(ok) => T::from_ok(ok), Err(error) => T::from_error(error), }) diff --git a/src/iter/unzip.rs b/src/iter/unzip.rs index 55e4b1f19..b5a669970 100644 --- a/src/iter/unzip.rs +++ b/src/iter/unzip.rs @@ -50,19 +50,15 @@ where // We have no idea what the consumers will look like for these // collections' `par_extend`, but we can intercept them in our own // `drive_unindexed`. Start with the left side, type `A`: - let iter = UnzipA { - base: pi, - op: op, - b: b, - }; + let iter = UnzipA { base: pi, op, b }; a.par_extend(iter); } /// Unzips the items of a parallel iterator into a pair of arbitrary /// `ParallelExtend` containers. /// -/// This is not directly public, but called by `ParallelIterator::unzip`. -pub fn unzip(pi: I) -> (FromA, FromB) +/// This is called by `ParallelIterator::unzip`. +pub(super) fn unzip(pi: I) -> (FromA, FromB) where I: ParallelIterator, FromA: Default + Send + ParallelExtend, @@ -75,8 +71,8 @@ where /// Unzip an `IndexedParallelIterator` into two arbitrary `Consumer`s. /// -/// This is not directly public, but called by `super::collect::unzip_into_vecs`. -pub fn unzip_indexed(pi: I, left: CA, right: CB) -> (CA::Result, CB::Result) +/// This is called by `super::collect::unzip_into_vecs`. +pub(super) fn unzip_indexed(pi: I, left: CA, right: CB) -> (CA::Result, CB::Result) where I: IndexedParallelIterator, CA: Consumer, @@ -86,8 +82,8 @@ where { let consumer = UnzipConsumer { op: &Unzip, - left: left, - right: right, + left, + right, }; pi.drive(consumer) } @@ -115,20 +111,15 @@ impl UnzipOp<(A, B)> for Unzip { /// Partitions the items of a parallel iterator into a pair of arbitrary /// `ParallelExtend` containers. /// -/// This is not directly public, but called by `ParallelIterator::partition`. -pub fn partition(pi: I, predicate: P) -> (A, B) +/// This is called by `ParallelIterator::partition`. +pub(super) fn partition(pi: I, predicate: P) -> (A, B) where I: ParallelIterator, A: Default + Send + ParallelExtend, B: Default + Send + ParallelExtend, P: Fn(&I::Item) -> bool + Sync + Send, { - execute( - pi, - Partition { - predicate: predicate, - }, - ) + execute(pi, Partition { predicate }) } /// An `UnzipOp` that routes items depending on a predicate function. @@ -160,8 +151,8 @@ where /// Partitions and maps the items of a parallel iterator into a pair of /// arbitrary `ParallelExtend` containers. /// -/// This is not directly public, but called by `ParallelIterator::partition_map`. -pub fn partition_map(pi: I, predicate: P) -> (A, B) +/// This called by `ParallelIterator::partition_map`. +pub(super) fn partition_map(pi: I, predicate: P) -> (A, B) where I: ParallelIterator, A: Default + Send + ParallelExtend, @@ -170,12 +161,7 @@ where L: Send, R: Send, { - execute( - pi, - PartitionMap { - predicate: predicate, - }, - ) + execute(pi, PartitionMap { predicate }) } /// An `UnzipOp` that routes items depending on how they are mapped `Either`. @@ -390,8 +376,8 @@ where let (left, right) = self.op.consume(item, self.left, self.right); UnzipFolder { op: self.op, - left: left, - right: right, + left, + right, } } diff --git a/src/iter/update.rs b/src/iter/update.rs index acfb28280..7bed5d9a4 100644 --- a/src/iter/update.rs +++ b/src/iter/update.rs @@ -18,21 +18,18 @@ pub struct Update { } impl Debug for Update { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("Update").field("base", &self.base).finish() } } -/// Create a new `Update` iterator. -/// -/// NB: a free fn because it is NOT part of the end-user API. -pub fn new(base: I, update_op: F) -> Update +impl Update where I: ParallelIterator, { - Update { - base: base, - update_op: update_op, + /// Create a new `Update` iterator. + pub(super) fn new(base: I, update_op: F) -> Self { + Update { base, update_op } } } @@ -78,7 +75,7 @@ where CB: ProducerCallback, { return self.base.with_producer(Callback { - callback: callback, + callback, update_op: self.update_op, }); @@ -99,7 +96,7 @@ where P: Producer, { let producer = UpdateProducer { - base: base, + base, update_op: &self.update_op, }; self.callback.callback(producer) @@ -173,10 +170,7 @@ struct UpdateConsumer<'f, C, F: 'f> { impl<'f, C, F> UpdateConsumer<'f, C, F> { fn new(base: C, update_op: &'f F) -> Self { - UpdateConsumer { - base: base, - update_op: update_op, - } + UpdateConsumer { base, update_op } } } @@ -282,12 +276,9 @@ where type Item = I::Item; fn next(&mut self) -> Option { - if let Some(mut v) = self.base.next() { - (self.update_op)(&mut v); - Some(v) - } else { - None - } + let mut v = self.base.next()?; + (self.update_op)(&mut v); + Some(v) } fn size_hint(&self) -> (usize, Option) { @@ -333,11 +324,8 @@ where F: FnMut(&mut I::Item), { fn next_back(&mut self) -> Option { - if let Some(mut v) = self.base.next_back() { - (self.update_op)(&mut v); - Some(v) - } else { - None - } + let mut v = self.base.next_back()?; + (self.update_op)(&mut v); + Some(v) } } diff --git a/src/iter/while_some.rs b/src/iter/while_some.rs index 4095d9a51..94ccdb323 100644 --- a/src/iter/while_some.rs +++ b/src/iter/while_some.rs @@ -15,14 +15,14 @@ pub struct WhileSome { base: I, } -/// Create a new `WhileSome` iterator. -/// -/// NB: a free fn because it is NOT part of the end-user API. -pub fn new(base: I) -> WhileSome +impl WhileSome where I: ParallelIterator, { - WhileSome { base: base } + /// Create a new `WhileSome` iterator. + pub(super) fn new(base: I) -> Self { + WhileSome { base } + } } impl ParallelIterator for WhileSome @@ -136,7 +136,7 @@ where false } }) - .map(|x| x.unwrap()), + .map(Option::unwrap), ); self } diff --git a/src/iter/zip.rs b/src/iter/zip.rs index 78f1dab0b..814222aa6 100644 --- a/src/iter/zip.rs +++ b/src/iter/zip.rs @@ -16,15 +16,15 @@ pub struct Zip { b: B, } -/// Create a new `Zip` iterator. -/// -/// NB: a free fn because it is NOT part of the end-user API. -pub fn new(a: A, b: B) -> Zip +impl Zip where A: IndexedParallelIterator, B: IndexedParallelIterator, { - Zip { a: a, b: b } + /// Create a new `Zip` iterator. + pub(super) fn new(a: A, b: B) -> Self { + Zip { a, b } + } } impl ParallelIterator for Zip @@ -67,7 +67,7 @@ where CB: ProducerCallback, { return self.a.with_producer(CallbackA { - callback: callback, + callback, b: self.b, }); @@ -87,10 +87,10 @@ where where A: Producer, { - return self.b.with_producer(CallbackB { - a_producer: a_producer, + self.b.with_producer(CallbackB { + a_producer, callback: self.callback, - }); + }) } } diff --git a/src/iter/zip_eq.rs b/src/iter/zip_eq.rs index e2af9fce3..e229e8aea 100644 --- a/src/iter/zip_eq.rs +++ b/src/iter/zip_eq.rs @@ -15,17 +15,16 @@ pub struct ZipEq { zip: Zip, } -/// Create a new `ZipEq` iterator. -/// -/// NB: a free fn because it is NOT part of the end-user API. -#[inline] -pub fn new(a: A, b: B) -> ZipEq +impl ZipEq where A: IndexedParallelIterator, B: IndexedParallelIterator, { - ZipEq { - zip: super::zip::new(a, b), + /// Create a new `ZipEq` iterator. + pub(super) fn new(a: A, b: B) -> Self { + ZipEq { + zip: super::Zip::new(a, b), + } } } diff --git a/src/lib.rs b/src/lib.rs index dd7bc2ea2..84bb2c3c8 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -1,6 +1,7 @@ -#![doc(html_root_url = "https://docs.rs/rayon/1.0")] +#![doc(html_root_url = "https://docs.rs/rayon/1.1")] #![deny(missing_debug_implementations)] #![deny(missing_docs)] +#![deny(unreachable_pub)] //! Data-parallelism library that makes it easy to convert sequential //! computations into parallel diff --git a/src/math.rs b/src/math.rs index cf8d12589..4968063fc 100644 --- a/src/math.rs +++ b/src/math.rs @@ -1,7 +1,7 @@ /// Divide `n` by `divisor`, and round up to the nearest integer /// if not evenly divisable. #[inline] -pub fn div_round_up(n: usize, divisor: usize) -> usize { +pub(super) fn div_round_up(n: usize, divisor: usize) -> usize { debug_assert!(divisor != 0, "Division by zero!"); if n == 0 { 0 diff --git a/src/option.rs b/src/option.rs index 417d54baf..0c021d93f 100644 --- a/src/option.rs +++ b/src/option.rs @@ -171,7 +171,7 @@ impl Producer for OptionProducer { /// /// If any item is `None`, then all previous items collected are discarded, /// and it returns only `None`. -impl<'a, C, T> FromParallelIterator> for Option +impl FromParallelIterator> for Option where C: FromParallelIterator, T: Send, diff --git a/src/range.rs b/src/range.rs index ada5aaa95..bd4c7dcec 100644 --- a/src/range.rs +++ b/src/range.rs @@ -215,9 +215,7 @@ indexed_range_impl! {isize} // other Range with just Iterator unindexed_range_impl! {u64, u64} unindexed_range_impl! {i64, u64} -#[cfg(has_i128)] unindexed_range_impl! {u128, u128} -#[cfg(has_i128)] unindexed_range_impl! {i128, u128} #[test] @@ -225,12 +223,11 @@ fn check_range_split_at_overflow() { // Note, this split index overflows i8! let producer = IterProducer { range: -100i8..100 }; let (left, right) = producer.split_at(150); - let r1: i32 = left.range.map(|i| i as i32).sum(); - let r2: i32 = right.range.map(|i| i as i32).sum(); + let r1: i32 = left.range.map(i32::from).sum(); + let r2: i32 = right.range.map(i32::from).sum(); assert_eq!(r1 + r2, -100); } -#[cfg(has_i128)] #[test] fn test_i128_len_doesnt_overflow() { use std::{i128, u128}; @@ -267,7 +264,6 @@ fn test_u64_opt_len() { } } -#[cfg(has_i128)] #[test] fn test_u128_opt_len() { use std::{u128, usize}; diff --git a/src/result.rs b/src/result.rs index 7fd7b6cac..054900f69 100644 --- a/src/result.rs +++ b/src/result.rs @@ -90,7 +90,7 @@ delegate_indexed_iterator! { /// If any item is `Err`, then all previous `Ok` items collected are /// discarded, and it returns that error. If there are multiple errors, the /// one returned is not deterministic. -impl<'a, C, T, E> FromParallelIterator> for Result +impl FromParallelIterator> for Result where C: FromParallelIterator, T: Send, diff --git a/src/slice/mergesort.rs b/src/slice/mergesort.rs index 6fb36f09f..45029c443 100644 --- a/src/slice/mergesort.rs +++ b/src/slice/mergesort.rs @@ -134,8 +134,8 @@ where { let len = v.len(); let v = v.as_mut_ptr(); - let v_mid = v.offset(mid as isize); - let v_end = v.offset(len as isize); + let v_mid = v.add(mid); + let v_end = v.add(len); // The merge process first copies the shorter run into `buf`. Then it traces the newly copied // run and the longer run forwards (or backwards), comparing their next unconsumed elements and @@ -161,7 +161,7 @@ where ptr::copy_nonoverlapping(v, buf, mid); hole = MergeHole { start: buf, - end: buf.offset(mid as isize), + end: buf.add(mid), dest: v, }; @@ -185,7 +185,7 @@ where ptr::copy_nonoverlapping(v_mid, buf, len - mid); hole = MergeHole { start: buf, - end: buf.offset((len - mid) as isize), + end: buf.add(len - mid), dest: v_mid, }; @@ -218,7 +218,7 @@ where impl Drop for MergeHole { fn drop(&mut self) { // `T` is not a zero-sized type, so it's okay to divide by its size. - let len = (self.end as usize - self.start as usize) / mem::size_of::(); + let len = (self.end as usize - self.start as usize) / size_of::(); unsafe { ptr::copy_nonoverlapping(self.start, self.dest, len); } @@ -355,7 +355,7 @@ where // Push this run onto the stack. runs.push(Run { - start: start, + start, len: end - start, }); end = start; @@ -469,10 +469,10 @@ where // remaining parts of `left` and `right` into `dest`. let mut s = State { left_start: left.as_mut_ptr(), - left_end: left.as_mut_ptr().offset(left_len as isize), + left_end: left.as_mut_ptr().add(left_len), right_start: right.as_mut_ptr(), - right_end: right.as_mut_ptr().offset(right_len as isize), - dest: dest, + right_end: right.as_mut_ptr().add(right_len), + dest, }; if left_len == 0 || right_len == 0 || left_len + right_len < MAX_SEQUENTIAL { @@ -500,7 +500,7 @@ where // Convert the pointers to `usize` because `*mut T` is not `Send`. let dest_l = dest as usize; - let dest_r = dest.offset((left_l.len() + right_l.len()) as isize) as usize; + let dest_r = dest.add(left_l.len() + right_l.len()) as usize; rayon_core::join( || par_merge(left_l, right_l, dest_l as *mut T, is_less), || par_merge(left_r, right_r, dest_r as *mut T, is_less), @@ -521,14 +521,14 @@ where impl Drop for State { fn drop(&mut self) { - let size = mem::size_of::(); + let size = size_of::(); let left_len = (self.left_end as usize - self.left_start as usize) / size; let right_len = (self.right_end as usize - self.right_start as usize) / size; // Copy array `left`, followed by `right`. unsafe { ptr::copy_nonoverlapping(self.left_start, self.dest, left_len); - self.dest = self.dest.offset(left_len as isize); + self.dest = self.dest.add(left_len); ptr::copy_nonoverlapping(self.right_start, self.dest, right_len); } } @@ -566,8 +566,8 @@ unsafe fn recurse( if into_buf { // Copy the chunk from `v` into `buf`. let (start, end) = chunks[0]; - let src = v.offset(start as isize); - let dest = buf.offset(start as isize); + let src = v.add(start); + let dest = buf.add(start); ptr::copy_nonoverlapping(src, dest, end - start); } return; @@ -594,8 +594,8 @@ unsafe fn recurse( // be executed, thus copying everything from `src` into `dest`. This way we ensure that all // chunks are in fact copied into `dest`, even if the merge process doesn't finish. let guard = CopyOnDrop { - src: src.offset(start as isize), - dest: dest.offset(start as isize), + src: src.add(start), + dest: dest.add(start), len: end - start, }; @@ -612,16 +612,16 @@ unsafe fn recurse( mem::forget(guard); // Merge chunks `(start, mid)` and `(mid, end)` from `src` into `dest`. - let src_left = slice::from_raw_parts_mut(src.offset(start as isize), mid - start); - let src_right = slice::from_raw_parts_mut(src.offset(mid as isize), end - mid); - par_merge(src_left, src_right, dest.offset(start as isize), is_less); + let src_left = slice::from_raw_parts_mut(src.add(start), mid - start); + let src_right = slice::from_raw_parts_mut(src.add(mid), end - mid); + par_merge(src_left, src_right, dest.add(start), is_less); } /// Sorts `v` using merge sort in parallel. /// /// The algorithm is stable, allocates memory, and `O(n log n)` worst-case. /// The allocated temporary buffer is of the same length as is `v`. -pub fn par_mergesort(v: &mut [T], is_less: F) +pub(super) fn par_mergesort(v: &mut [T], is_less: F) where T: Send, F: Fn(&T, &T) -> bool + Sync, @@ -678,7 +678,7 @@ where let l = CHUNK_LENGTH * i; let r = l + chunk.len(); unsafe { - let buf = (buf as *mut T).offset(l as isize); + let buf = (buf as *mut T).add(l); (l, r, mergesort(chunk, buf, &is_less)) } }) @@ -716,7 +716,7 @@ where // All chunks are properly sorted. // Now we just have to merge them together. unsafe { - recurse(v.as_mut_ptr(), buf as *mut T, &chunks, false, &is_less); + recurse(v.as_mut_ptr(), buf, &chunks, false, &is_less); } } diff --git a/src/slice/mod.rs b/src/slice/mod.rs index c1bd46787..d78786996 100644 --- a/src/slice/mod.rs +++ b/src/slice/mod.rs @@ -40,13 +40,13 @@ pub trait ParallelSlice { /// .min(); /// assert_eq!(Some(&1), smallest); /// ``` - fn par_split

(&self, separator: P) -> Split + fn par_split

(&self, separator: P) -> Split<'_, T, P> where P: Fn(&T) -> bool + Sync + Send, { Split { slice: self.as_parallel_slice(), - separator: separator, + separator, } } @@ -60,9 +60,9 @@ pub trait ParallelSlice { /// let windows: Vec<_> = [1, 2, 3].par_windows(2).collect(); /// assert_eq!(vec![[1, 2], [2, 3]], windows); /// ``` - fn par_windows(&self, window_size: usize) -> Windows { + fn par_windows(&self, window_size: usize) -> Windows<'_, T> { Windows { - window_size: window_size, + window_size, slice: self.as_parallel_slice(), } } @@ -81,10 +81,10 @@ pub trait ParallelSlice { /// let chunks: Vec<_> = [1, 2, 3, 4, 5].par_chunks(2).collect(); /// assert_eq!(chunks, vec![&[1, 2][..], &[3, 4], &[5]]); /// ``` - fn par_chunks(&self, chunk_size: usize) -> Chunks { + fn par_chunks(&self, chunk_size: usize) -> Chunks<'_, T> { assert!(chunk_size != 0, "chunk_size must not be zero"); Chunks { - chunk_size: chunk_size, + chunk_size, slice: self.as_parallel_slice(), } } @@ -115,13 +115,13 @@ pub trait ParallelSliceMut { /// .for_each(|slice| slice.reverse()); /// assert_eq!(array, [3, 2, 1, 0, 8, 4, 2, 0, 9, 6, 3]); /// ``` - fn par_split_mut

(&mut self, separator: P) -> SplitMut + fn par_split_mut

(&mut self, separator: P) -> SplitMut<'_, T, P> where P: Fn(&T) -> bool + Sync + Send, { SplitMut { slice: self.as_parallel_slice_mut(), - separator: separator, + separator, } } @@ -141,10 +141,10 @@ pub trait ParallelSliceMut { /// .for_each(|slice| slice.reverse()); /// assert_eq!(array, [2, 1, 4, 3, 5]); /// ``` - fn par_chunks_mut(&mut self, chunk_size: usize) -> ChunksMut { + fn par_chunks_mut(&mut self, chunk_size: usize) -> ChunksMut<'_, T> { assert!(chunk_size != 0, "chunk_size must not be zero"); ChunksMut { - chunk_size: chunk_size, + chunk_size, slice: self.as_parallel_slice_mut(), } } @@ -497,7 +497,7 @@ impl<'data, T: 'data + Sync> Producer for IterProducer<'data, T> { type IntoIter = ::std::slice::Iter<'data, T>; fn into_iter(self) -> Self::IntoIter { - self.slice.into_iter() + self.slice.iter() } fn split_at(self, index: usize) -> (Self, Self) { @@ -718,7 +718,7 @@ impl<'data, T: 'data + Send> Producer for IterMutProducer<'data, T> { type IntoIter = ::std::slice::IterMut<'data, T>; fn into_iter(self) -> Self::IntoIter { - self.slice.into_iter() + self.slice.iter_mut() } fn split_at(self, index: usize) -> (Self, Self) { @@ -820,7 +820,7 @@ impl<'data, T, P: Clone> Clone for Split<'data, T, P> { } impl<'data, T: Debug, P> Debug for Split<'data, T, P> { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("Split").field("slice", &self.slice).finish() } } @@ -887,7 +887,7 @@ pub struct SplitMut<'data, T: 'data, P> { } impl<'data, T: Debug, P> Debug for SplitMut<'data, T, P> { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("SplitMut") .field("slice", &self.slice) .finish() diff --git a/src/slice/quicksort.rs b/src/slice/quicksort.rs index 8cfd39a11..4fd781316 100644 --- a/src/slice/quicksort.rs +++ b/src/slice/quicksort.rs @@ -175,7 +175,7 @@ where F: Fn(&T, &T) -> bool, { for i in 1..v.len() { - shift_tail(&mut v[..i + 1], is_less); + shift_tail(&mut v[..=i], is_less); } } @@ -259,7 +259,7 @@ where let mut offsets_l: [u8; BLOCK] = unsafe { mem::uninitialized() }; // The current block on the right side (from `r.offset(-block_r)` to `r`). - let mut r = unsafe { l.offset(v.len() as isize) }; + let mut r = unsafe { l.add(v.len()) }; let mut block_r = BLOCK; let mut start_r = ptr::null_mut(); let mut end_r = ptr::null_mut(); @@ -367,12 +367,12 @@ where if start_l == end_l { // All out-of-order elements in the left block were moved. Move to the next block. - l = unsafe { l.offset(block_l as isize) }; + l = unsafe { l.add(block_l) }; } if start_r == end_r { // All out-of-order elements in the right block were moved. Move to the previous block. - r = unsafe { r.offset(-(block_r as isize)) }; + r = unsafe { r.sub(block_r) }; } if is_done { @@ -543,7 +543,7 @@ fn break_patterns(v: &mut [T]) { if mem::size_of::() <= 4 { gen_u32() as usize } else { - (((gen_u32() as u64) << 32) | (gen_u32() as u64)) as usize + ((u64::from(gen_u32()) << 32) | u64::from(gen_u32())) as usize } }; @@ -743,7 +743,7 @@ where /// Sorts `v` using pattern-defeating quicksort in parallel. /// /// The algorithm is unstable, in-place, and `O(n log n)` worst-case. -pub fn par_quicksort(v: &mut [T], is_less: F) +pub(super) fn par_quicksort(v: &mut [T], is_less: F) where T: Send, F: Fn(&T, &T) -> bool + Sync, diff --git a/src/slice/test.rs b/src/slice/test.rs index 94a72140a..d335318f1 100644 --- a/src/slice/test.rs +++ b/src/slice/test.rs @@ -81,9 +81,9 @@ macro_rules! sort { [(); 10].$f(|a, b| a.cmp(b)); [(); 100].$f(|a, b| a.cmp(b)); - let mut v = [0xDEADBEEFu64]; + let mut v = [0xDEAD_BEEFu64]; v.$f(|a, b| a.cmp(b)); - assert!(v == [0xDEADBEEF]); + assert!(v == [0xDEAD_BEEF]); } }; } diff --git a/src/split_producer.rs b/src/split_producer.rs index 1a6c83c15..36d1b70f2 100644 --- a/src/split_producer.rs +++ b/src/split_producer.rs @@ -5,7 +5,7 @@ use iter::plumbing::{Folder, UnindexedProducer}; /// Common producer for splitting on a predicate. -pub struct SplitProducer<'p, P: 'p, V> { +pub(super) struct SplitProducer<'p, P: 'p, V> { data: V, separator: &'p P, @@ -14,7 +14,7 @@ pub struct SplitProducer<'p, P: 'p, V> { } /// Helper trait so `&str`, `&[T]`, and `&mut [T]` can share `SplitProducer`. -pub trait Fissile

: Sized { +pub(super) trait Fissile

: Sized { fn length(&self) -> usize; fn midpoint(&self, end: usize) -> usize; fn find(&self, separator: &P, start: usize, end: usize) -> Option; @@ -30,17 +30,17 @@ impl<'p, P, V> SplitProducer<'p, P, V> where V: Fissile

+ Send, { - pub fn new(data: V, separator: &'p P) -> Self { + pub(super) fn new(data: V, separator: &'p P) -> Self { SplitProducer { tail: data.length(), - data: data, - separator: separator, + data, + separator, } } /// Common `fold_with` implementation, integrating `SplitTerminator`'s /// need to sometimes skip its final empty item. - pub fn fold_with(self, folder: F, skip_last: bool) -> F + pub(super) fn fold_with(self, folder: F, skip_last: bool) -> F where F: Folder, { diff --git a/src/str.rs b/src/str.rs index b19f52cac..a60555dce 100644 --- a/src/str.rs +++ b/src/str.rs @@ -70,7 +70,7 @@ pub trait ParallelString { /// let max = "hello".par_chars().max_by_key(|c| *c as i32); /// assert_eq!(Some('o'), max); /// ``` - fn par_chars(&self) -> Chars { + fn par_chars(&self) -> Chars<'_> { Chars { chars: self.as_parallel_string(), } @@ -85,7 +85,7 @@ pub trait ParallelString { /// let min = "hello".par_char_indices().min_by_key(|&(_i, c)| c as i32); /// assert_eq!(Some((1, 'e')), min); /// ``` - fn par_char_indices(&self) -> CharIndices { + fn par_char_indices(&self) -> CharIndices<'_> { CharIndices { chars: self.as_parallel_string(), } @@ -105,7 +105,7 @@ pub trait ParallelString { /// let max = "hello".par_bytes().max(); /// assert_eq!(Some(b'o'), max); /// ``` - fn par_bytes(&self) -> Bytes { + fn par_bytes(&self) -> Bytes<'_> { Bytes { chars: self.as_parallel_string(), } @@ -129,7 +129,7 @@ pub trait ParallelString { /// let utf16_len = text.par_encode_utf16().count(); /// assert!(utf16_len <= utf8_len); /// ``` - fn par_encode_utf16(&self) -> EncodeUtf16 { + fn par_encode_utf16(&self) -> EncodeUtf16<'_> { EncodeUtf16 { chars: self.as_parallel_string(), } @@ -151,7 +151,7 @@ pub trait ParallelString { /// .sum(); /// assert_eq!(10, total); /// ``` - fn par_split(&self, separator: P) -> Split

{ + fn par_split(&self, separator: P) -> Split<'_, P> { Split::new(self.as_parallel_string(), separator) } @@ -172,7 +172,7 @@ pub trait ParallelString { /// .collect(); /// assert_eq!(vec!["", "", "1 + 3", " * 2"], parts); /// ``` - fn par_split_terminator(&self, terminator: P) -> SplitTerminator

{ + fn par_split_terminator(&self, terminator: P) -> SplitTerminator<'_, P> { SplitTerminator::new(self.as_parallel_string(), terminator) } @@ -191,7 +191,7 @@ pub trait ParallelString { /// .collect(); /// assert_eq!(vec![11, 7], lengths); /// ``` - fn par_lines(&self) -> Lines { + fn par_lines(&self) -> Lines<'_> { Lines(self.as_parallel_string()) } @@ -210,7 +210,7 @@ pub trait ParallelString { /// .max_by_key(|word| word.len()); /// assert_eq!(Some("longest"), longest); /// ``` - fn par_split_whitespace(&self) -> SplitWhitespace { + fn par_split_whitespace(&self) -> SplitWhitespace<'_> { SplitWhitespace(self.as_parallel_string()) } @@ -230,10 +230,10 @@ pub trait ParallelString { /// .sum(); /// assert_eq!(10, total); /// ``` - fn par_matches(&self, pattern: P) -> Matches

{ + fn par_matches(&self, pattern: P) -> Matches<'_, P> { Matches { chars: self.as_parallel_string(), - pattern: pattern, + pattern, } } @@ -252,10 +252,10 @@ pub trait ParallelString { /// .collect(); /// assert_eq!(digits, vec![(0, "1"), (3, "2"), (14, "3"), (17, "4")]); /// ``` - fn par_match_indices(&self, pattern: P) -> MatchIndices

{ + fn par_match_indices(&self, pattern: P) -> MatchIndices<'_, P> { MatchIndices { chars: self.as_parallel_string(), - pattern: pattern, + pattern, } } } @@ -282,16 +282,16 @@ mod private { /// Implementing this trait is not permitted outside of `rayon`. pub trait Pattern: Sized + Sync + Send { private_decl! {} - fn find_in(&self, &str) -> Option; - fn rfind_in(&self, &str) -> Option; - fn is_suffix_of(&self, &str) -> bool; - fn fold_splits<'ch, F>(&self, &'ch str, folder: F, skip_last: bool) -> F + fn find_in(&self, haystack: &str) -> Option; + fn rfind_in(&self, haystack: &str) -> Option; + fn is_suffix_of(&self, haystack: &str) -> bool; + fn fold_splits<'ch, F>(&self, haystack: &'ch str, folder: F, skip_last: bool) -> F where F: Folder<&'ch str>; - fn fold_matches<'ch, F>(&self, &'ch str, folder: F) -> F + fn fold_matches<'ch, F>(&self, haystack: &'ch str, folder: F) -> F where F: Folder<&'ch str>; - fn fold_match_indices<'ch, F>(&self, &'ch str, folder: F, base: usize) -> F + fn fold_match_indices<'ch, F>(&self, haystack: &'ch str, folder: F, base: usize) -> F where F: Folder<(usize, &'ch str)>; } @@ -582,10 +582,7 @@ pub struct Split<'ch, P: Pattern> { impl<'ch, P: Pattern> Split<'ch, P> { fn new(chars: &'ch str, separator: P) -> Self { - Split { - chars: chars, - separator: separator, - } + Split { chars, separator } } } @@ -651,10 +648,7 @@ struct SplitTerminatorProducer<'ch, 'sep, P: Pattern + 'sep> { impl<'ch, P: Pattern> SplitTerminator<'ch, P> { fn new(chars: &'ch str, terminator: P) -> Self { - SplitTerminator { - chars: chars, - terminator: terminator, - } + SplitTerminator { chars, terminator } } } @@ -690,7 +684,7 @@ impl<'ch, 'sep, P: Pattern + 'sep> UnindexedProducer for SplitTerminatorProducer self.skip_last = false; SplitTerminatorProducer { splitter: right, - skip_last: skip_last, + skip_last, } }); (self, right) diff --git a/src/vec.rs b/src/vec.rs index dfa7cc4a1..767bf8ca0 100644 --- a/src/vec.rs +++ b/src/vec.rs @@ -66,7 +66,7 @@ impl IndexedParallelIterator for IntoIter { let mut slice = self.vec.as_mut_slice(); slice = std::slice::from_raw_parts_mut(slice.as_mut_ptr(), len); - callback.callback(VecProducer { slice: slice }) + callback.callback(VecProducer { slice }) } } } @@ -116,7 +116,8 @@ impl<'data, T: 'data> Iterator for SliceDrain<'data, T> { type Item = T; fn next(&mut self) -> Option { - self.iter.next().map(|ptr| unsafe { std::ptr::read(ptr) }) + let ptr = self.iter.next()?; + Some(unsafe { std::ptr::read(ptr) }) } fn size_hint(&self) -> (usize, Option) { @@ -127,9 +128,8 @@ impl<'data, T: 'data> Iterator for SliceDrain<'data, T> { impl<'data, T: 'data> DoubleEndedIterator for SliceDrain<'data, T> { fn next_back(&mut self) -> Option { - self.iter - .next_back() - .map(|ptr| unsafe { std::ptr::read(ptr) }) + let ptr = self.iter.next_back()?; + Some(unsafe { std::ptr::read(ptr) }) } } diff --git a/tests/debug.rs b/tests/debug.rs index 8ea331713..24cfd9344 100644 --- a/tests/debug.rs +++ b/tests/debug.rs @@ -121,8 +121,8 @@ fn debug_adaptors() { check(v.par_iter().cloned()); check(v.par_iter().enumerate()); check(v.par_iter().filter(|_| true)); - check(v.par_iter().filter_map(|x| Some(x))); - check(v.par_iter().flat_map(|x| Some(x))); + check(v.par_iter().filter_map(Some)); + check(v.par_iter().flat_map(Some)); check(v.par_iter().map(Some).flatten()); check(v.par_iter().fold(|| 0, |x, _| x)); check(v.par_iter().fold_with(0, |x, _| x)); diff --git a/tests/named-threads.rs b/tests/named-threads.rs index f0868c5d4..408b13536 100644 --- a/tests/named-threads.rs +++ b/tests/named-threads.rs @@ -16,7 +16,7 @@ fn named_threads() { let thread_names = (0..N) .into_par_iter() - .flat_map(|_| ::std::thread::current().name().map(|s| s.to_owned())) + .flat_map(|_| ::std::thread::current().name().map(str::to_owned)) .collect::>(); let all_contains_name = thread_names diff --git a/tests/octillion.rs b/tests/octillion.rs index 878c2d36a..1949155bb 100644 --- a/tests/octillion.rs +++ b/tests/octillion.rs @@ -14,19 +14,19 @@ fn octillion_flat() -> impl ParallelIterator { (0u32..1_000_000_000) .into_par_iter() .with_max_len(1_000) - .map(|i| i as u64 * 1_000_000_000) + .map(|i| u64::from(i) * 1_000_000_000) .flat_map(|i| { (0u32..1_000_000_000) .into_par_iter() .with_max_len(1_000) - .map(move |j| i + j as u64) + .map(move |j| i + u64::from(j)) }) - .map(|i| i as u128 * 1_000_000_000) + .map(|i| u128::from(i) * 1_000_000_000) .flat_map(|i| { (0u32..1_000_000_000) .into_par_iter() .with_max_len(1_000) - .map(move |j| i + j as u128) + .map(move |j| i + u128::from(j)) }) } diff --git a/tests/producer_split_at.rs b/tests/producer_split_at.rs index 0ed18e2da..812eceb7c 100644 --- a/tests/producer_split_at.rs +++ b/tests/producer_split_at.rs @@ -113,7 +113,7 @@ fn check_len(iter: &I, len: usize) { #[test] fn empty() { let v = vec![42]; - check(&v[..0], || rayon::iter::empty()); + check(&v[..0], rayon::iter::empty); } #[test] diff --git a/tests/sort-panic-safe.rs b/tests/sort-panic-safe.rs index ab200bec3..c216c46e9 100644 --- a/tests/sort-panic-safe.rs +++ b/tests/sort-panic-safe.rs @@ -11,12 +11,9 @@ use std::cmp::{self, Ordering}; use std::panic; use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering::Relaxed; -#[allow(deprecated)] -use std::sync::atomic::ATOMIC_USIZE_INIT; use std::thread; -#[allow(deprecated)] -static VERSIONS: AtomicUsize = ATOMIC_USIZE_INIT; +static VERSIONS: AtomicUsize = AtomicUsize::new(0); lazy_static! { static ref DROP_COUNTS: Vec = (0..20_000).map(|_| AtomicUsize::new(0)).collect(); @@ -128,7 +125,7 @@ thread_local!(static SILENCE_PANIC: Cell = Cell::new(false)); fn sort_panic_safe() { let prev = panic::take_hook(); panic::set_hook(Box::new(move |info| { - if !SILENCE_PANIC.with(|s| s.get()) { + if !SILENCE_PANIC.with(Cell::get) { prev(info); } })); @@ -141,7 +138,7 @@ fn sort_panic_safe() { let mut input = (0..len) .map(|id| DropCounter { x: rng.gen_range(0, modulus), - id: id, + id, version: Cell::new(0), }) .collect::>();