1
  2
  3
  4
  5
  6
  7
  8
  9
 10
 11
 12
 13
 14
 15
 16
 17
 18
 19
 20
 21
 22
 23
 24
 25
 26
 27
 28
 29
 30
 31
 32
 33
 34
 35
 36
 37
 38
 39
 40
 41
 42
 43
 44
 45
 46
 47
 48
 49
 50
 51
 52
 53
 54
 55
 56
 57
 58
 59
 60
 61
 62
 63
 64
 65
 66
 67
 68
 69
 70
 71
 72
 73
 74
 75
 76
 77
 78
 79
 80
 81
 82
 83
 84
 85
 86
 87
 88
 89
 90
 91
 92
 93
 94
 95
 96
 97
 98
 99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
#![allow(non_snake_case, clippy::type_complexity, irrefutable_let_patterns, clippy::new_without_default, unused_mut, unreachable_code)]

use futures::{pin_mut, ready, stream, Stream, StreamExt};
use pin_project::pin_project;
use serde::{Deserialize, Serialize};
use std::{
	pin::Pin, task::{Context, Poll}
};
use sum::*;

use super::{
	DistributedPipe, DistributedSink, Factory, ParallelPipe, ParallelSink, PipeTask, PipeTaskAsync, Reducer, ReducerAsync, ReducerProcessSend, ReducerSend
};
use crate::{
	pool::ProcessSend, sink::{Sink, SinkMap}
};

fn substream<'a, 'b, S, F1, F2, O>(
	mut stream: Pin<&'a mut Peekable<'b, S>>, mut is: F1, mut unwrap: F2,
) -> impl Stream<Item = F2::Output> + 'a
where
	S: Stream,
	F1: FnMut(&S::Item) -> bool + 'a,
	F2: FnMut(S::Item) -> O + 'a,
	'a: 'b,
{
	stream::poll_fn(move |cx| match ready!(stream.as_mut().poll_peek(cx)) {
		Some(enum_) if is(enum_) => Poll::Ready(Some(
			if let Poll::Ready(Some(enum_)) = stream.as_mut().poll_next(cx) {
				unwrap(enum_)
			} else {
				unreachable!()
			},
		)),
		Some(_) => Poll::Pending,
		None => Poll::Ready(None),
	})
	.fuse()
}

macro_rules! impl_tuple {
	($reduceafactory:ident $reducea:ident $reduceaasync:ident $reducebfactory:ident $reduceb:ident $reducebasync:ident $async:ident $enum:ident $($copy:ident)? : $($i:ident $r:ident $o:ident $c:ident $iterator:ident $reducera:ident $reducerb:ident $num:tt $t:ident $($copyb:ident)? , $comma:tt)*) => (
		impl<
				Source,
				$($r: ParallelSink<Source, Output = $o>,)*
				$($o,)*
			> ParallelSink<Source> for ($($r,)*)
				where Source: $($copy)*,
		{
			type Output = ($($o,)*);
			type Pipe = ($($r::Pipe,)*);
			type ReduceAFactory = $reduceafactory<$($r::ReduceAFactory,)*>;
			type ReduceA = $reducea<$($r::ReduceA,)*>;
			type ReduceC = $reduceb<$($r::ReduceC,)*>;

			fn reducers(self) -> (Self::Pipe, Self::ReduceAFactory, Self::ReduceC) {
				$(let ($iterator, $reducera, $t) = self.$num.reducers();)*
				(
					($($iterator,)*),
					$reduceafactory($($reducera,)*),
					$reduceb{$($t,)*},
				)
			}
		}
		impl<
				Source,
				$($r: DistributedSink<Source, Output = $o>,)*
				$($o,)*
			> DistributedSink<Source> for ($($r,)*)
				where Source: $($copy)*,
		{
			type Output = ($($o,)*);
			type Pipe = ($($r::Pipe,)*);
			type ReduceAFactory = $reduceafactory<$($r::ReduceAFactory,)*>;
			type ReduceBFactory = $reducebfactory<$($r::ReduceBFactory,)*>;
			type ReduceA = $reducea<$($r::ReduceA,)*>;
			type ReduceB = $reduceb<$($r::ReduceB,)*>;
			type ReduceC = $reduceb<$($r::ReduceC,)*>;

			fn reducers(self) -> (Self::Pipe, Self::ReduceAFactory, Self::ReduceBFactory, Self::ReduceC) {
				$(let ($iterator, $reducera, $reducerb, $t) = self.$num.reducers();)*
				(
					($($iterator,)*),
					$reduceafactory($($reducera,)*),
					$reducebfactory($($reducerb,)*),
					$reduceb{$($t,)*},
				)
			}
		}

		impl<Source, $($i: ParallelPipe<Source>,)*>
			ParallelPipe<Source> for ($($i,)*)
				where Source: $($copy)*,
		{
			type Item = $enum<$($i::Item,)*>;
			type Task = ($($i::Task,)*);

			fn task(&self) -> Self::Task {
				($(self.$num.task(),)*)
			}
		}
		impl<Source, $($i: DistributedPipe<Source>,)*>
			DistributedPipe<Source> for ($($i,)*)
				where Source: $($copy)*,
		{
			type Item = $enum<$($i::Item,)*>;
			type Task = ($($i::Task,)*);

			fn task(&self) -> Self::Task {
				($(self.$num.task(),)*)
			}
		}

		impl<Source, $($c: PipeTask<Source>,)*> PipeTask<Source> for ($($c,)*)
		where
			Source: $($copy)*,
		{
			type Item = $enum<$($c::Item,)*>;
			type Async = $async<Source, $($c::Async,)*>;
			fn into_async(self) -> Self::Async {
				$async{
					$($t: self.$num.into_async(),)*
					pending: None,
					given: ($(false $comma)*),
				}
			}
		}

		#[pin_project]
		pub struct $async<Source, $($c,)*> {
			$(#[pin] $t: $c,)*
			pending: Option<Option<Source>>,
			given: ($(bool $comma)*),
		}

		#[allow(unused_variables)]
		impl<Source, $($c: PipeTaskAsync<Source>,)*> PipeTaskAsync<Source> for $async<Source, $($c,)*>
		where
			Source: $($copy)*,
		{
			type Item = $enum<$($c::Item,)*>;

			#[allow(non_snake_case)]
			fn poll_run(
				self: Pin<&mut Self>, cx: &mut Context, mut stream: Pin<&mut impl Stream<Item = Source>>,
				mut sink: Pin<&mut impl Sink<Item = Self::Item>>,
			) -> Poll<()> {
				let mut self_ = self.project();
				// buffer, copy to each
				let mut ready = ($(false $comma)*);
				loop {
					let mut progress = false;
					if self_.pending.is_none() {
						*self_.pending = match stream.as_mut().poll_next(cx) { Poll::Ready(x) => Some(x), Poll::Pending => None};
					}
					$(
						if !ready.$num {
							let sink_ = SinkMap::new(sink.as_mut(), $enum::$t);
							pin_mut!(sink_);
							let pending = &mut self_.pending;
							let given = &mut self_.given.$num;
							let stream_ = stream::poll_fn(|cx| {
								if !*given && pending.is_some() {
									*given = true;
									progress = true;
									$(
										return Poll::Ready(*pending.as_ref().unwrap());
										let $copyb = ();
									)?
									Poll::Ready(pending.take().unwrap().take())
								} else {
									Poll::Pending
								}
							}).fuse();
							pin_mut!(stream_);
							ready.$num = self_.$t.as_mut().poll_run(cx, stream_, sink_).is_ready();
							if ready.$num {
								self_.given.$num = true;
							}
						}
					)*
					if $(ready.$num &&)* true {
						break Poll::Ready(());
					}
					if !progress {
						break Poll::Pending;
					}
					if $(self_.given.$num &&)* true {
						$(self_.given.$num = false;)*
						*self_.pending = None;
					}
				}
			}
		}

		#[derive(Clone, Serialize, Deserialize)]
		pub struct $reduceafactory<$($r,)*>($(pub(crate) $r,)*);
		impl<$($r:Factory,)*> Factory for $reduceafactory<$($r,)*> {
			type Item = $reducea<$($r::Item,)*>;

			fn make(&self) -> Self::Item {
				$reducea{
					$($t: self.$num.make(),)*
				}
			}
		}

		#[derive(Serialize, Deserialize)]
		pub struct $reducea<$($t,)*> {
			$($t: $t,)*
		}
		impl<$($t: Reducer,)*> Reducer for $reducea<$($t,)*> {
			type Item = $enum<$($t::Item,)*>;
			type Output = ($($t::Output,)*);
			type Async = $reduceaasync<$($t::Async,)*>;

			fn into_async(self) -> Self::Async {
				$reduceaasync{
					$($t: self.$t.into_async(),)*
					peeked: None,
					output: ($(None::<$t::Output>,)*),
				}
			}
		}
		#[pin_project]
		pub struct $reduceaasync<$($t,)*> where $($t: ReducerAsync,)* {
			$(#[pin] $t: $t,)*
			peeked: Option<$enum<$($t::Item,)*>>,
			output: ($(Option<$t::Output>,)*),
		}
		#[allow(unused_variables)]
		impl<$($t: ReducerAsync,)*> ReducerAsync for $reduceaasync<$($t,)*> {
			type Item = $enum<$($t::Item,)*>;
			type Output = ($($t::Output,)*);

			fn poll_forward(self: Pin<&mut Self>, cx: &mut Context, mut stream: Pin<&mut impl Stream<Item = Self::Item>>) -> Poll<()> {
				let mut self_ = self.project();
				let mut ready = ($(false $comma)*);
				loop {
					let mut progress = false;
					$(if !ready.$num {
						let stream = Peekable{stream:stream.as_mut(),peeked:&mut *self_.peeked};
						pin_mut!(stream);
						let stream_ = substream(stream, |item| if let $enum::$t(_) = item { true } else { false }, |item| { progress = true; if let $enum::$t(item) = item { item } else { unreachable!() } });
						pin_mut!(stream_);
						ready.$num = self_.$t.as_mut().poll_forward(cx, stream_).is_ready();
					})*
					if $(ready.$num &&)* true {
						break Poll::Ready(())
					}
					if !progress {
						break Poll::Pending;
					}
				}
			}
			fn poll_output(self: Pin<&mut Self>, cx: &mut Context) -> Poll<Self::Output> {
				let mut self_ = self.project();
				let mut pending = false;
				$(
					if self_.output.$num.is_none() {
						if let Poll::Ready(output) = self_.$t.poll_output(cx) {
							self_.output.$num = Some(output);
						} else {
							pending = true;
						}
					}
				)*
				if !pending {
					Poll::Ready(($(self_.output.$num.take().unwrap(),)*))
				} else {
					Poll::Pending
				}
			}
		}
		impl<$($t: Reducer,)*> ReducerProcessSend for $reducea<$($t,)*> where $($t::Output: ProcessSend + 'static,)* {
			type Output = ($($t::Output,)*);
		}
		impl<$($t: Reducer,)*> ReducerSend for $reducea<$($t,)*> where $($t::Output: Send + 'static,)* {
			type Output = ($($t::Output,)*);
		}

		pub struct $reducebfactory<$($r,)*>($(pub(crate) $r,)*);
		impl<$($r:Factory,)*> Factory for $reducebfactory<$($r,)*> {
			type Item = $reduceb<$($r::Item,)*>;

			fn make(&self) -> Self::Item {
				$reduceb{
					$($t: self.$num.make(),)*
				}
			}
		}

		#[derive(Serialize, Deserialize)]
		pub struct $reduceb<$($t,)*> {
			$($t: $t,)*
		}
		impl<$($t,)*> $reduceb<$($t,)*> {
			#[allow(clippy::too_many_arguments)]
			pub fn new($($t: $t,)*) -> Self {
				Self {
					$($t,)*
				}
			}
		}
		impl<$($t: Reducer,)*> Reducer for $reduceb<$($t,)*> {
			type Item = ($($t::Item,)*);
			type Output = ($($t::Output,)*);
			type Async = $reducebasync<$($t::Async,)*>;

			fn into_async(self) -> Self::Async {
				$reducebasync{
					$($t: self.$t.into_async(),)*
					peeked: None,
					output: ($(None::<$t::Output>,)*),
				}
			}
		}
		impl<$($t: ReducerProcessSend,)*> ReducerProcessSend for $reduceb<$($t,)*> {
			type Output = ($(<$t as ReducerProcessSend>::Output,)*);
		}
		impl<$($t: ReducerSend,)*> ReducerSend for $reduceb<$($t,)*> {
			type Output = ($(<$t as ReducerSend>::Output,)*);
		}
		#[pin_project]
		pub struct $reducebasync<$($t,)*> where $($t: ReducerAsync,)* {
			$(#[pin] $t: $t,)*
			peeked: Option<($(Option<$t::Item>,)*)>,
			output: ($(Option<$t::Output>,)*),
		}
		#[allow(unused_variables)]
		impl<$($t: ReducerAsync,)*> ReducerAsync for $reducebasync<$($t,)*> {
			type Item = ($($t::Item,)*);
			type Output = ($($t::Output,)*);

			fn poll_forward(self: Pin<&mut Self>, cx: &mut Context, stream: Pin<&mut impl Stream<Item = Self::Item>>) -> Poll<()> {
				let mut self_ = self.project();
				let mut ready = ($(false $comma)*);
				let stream = stream.map(|item| ($(Some(item.$num),)*));
				pin_mut!(stream);
				loop {
					let mut progress = false;
					$(if !ready.$num {
						let stream = Peekable{stream:stream.as_mut(),peeked:&mut *self_.peeked};
						pin_mut!(stream);
						let stream = stream::poll_fn(|cx| match ready!(stream.as_mut().poll_peek(cx)) {
							Some(enum_) if enum_.$num.is_some() => {
								let ret = enum_.$num.take().unwrap();
								progress = true;
								Poll::Ready(Some(ret))
							}
							Some(_) => {
								Poll::Pending
							},
							None => Poll::Ready(None),
						}).fuse();
						pin_mut!(stream);
						ready.$num = self_.$t.as_mut().poll_forward(cx, stream).is_ready();
					})*
					if let Some(peeked) = self_.peeked {
						if $(peeked.$num.is_none() &&)* true {
							*self_.peeked = None;
							progress = true;
						}
					}
					if $(ready.$num &&)* true {
						break Poll::Ready(());
					}
					if !progress {
						break Poll::Pending;
					}
				}
			}
			fn poll_output(self: Pin<&mut Self>, cx: &mut Context) -> Poll<Self::Output> {
				let mut self_ = self.project();
				let mut pending = false;
				$(
					if self_.output.$num.is_none() {
						if let Poll::Ready(output) = self_.$t.poll_output(cx) {
							self_.output.$num = Some(output);
						} else {
							pending = true;
						}
					}
				)*
				if !pending {
					Poll::Ready(($(self_.output.$num.take().unwrap(),)*))
				} else {
					Poll::Pending
				}
			}
		}
	);
}
impl_tuple!(ReduceA0Factory ReduceA0 ReduceA0Async ReduceC0Factory ReduceC0 ReduceC0Async AsyncTuple0 Sum0:);
impl_tuple!(ReduceA1Factory ReduceA1 ReduceA1Async ReduceC1Factory ReduceC1 ReduceC1Async AsyncTuple1 Sum1: I0 R0 O0 C0 iterator_0 reducer_a_0 reducer_b_0 0 A,,);
impl_tuple!(ReduceA2Factory ReduceA2 ReduceA2Async ReduceC2Factory ReduceC2 ReduceC2Async AsyncTuple2 Sum2 Copy: I0 R0 O0 C0 iterator_0 reducer_a_0 reducer_b_0 0 A Copy,, I1 R1 O1 C1 iterator_1 reducer_a_1 reducer_b_1 1 B Copy,,);
impl_tuple!(ReduceA3Factory ReduceA3 ReduceA3Async ReduceC3Factory ReduceC3 ReduceC3Async AsyncTuple3 Sum3 Copy: I0 R0 O0 C0 iterator_0 reducer_a_0 reducer_b_0 0 A Copy,, I1 R1 O1 C1 iterator_1 reducer_a_1 reducer_b_1 1 B Copy,, I2 R2 O2 C2 iterator_2 reducer_a_2 reducer_b_2 2 C Copy,,);
impl_tuple!(ReduceA4Factory ReduceA4 ReduceA4Async ReduceC4Factory ReduceC4 ReduceC4Async AsyncTuple4 Sum4 Copy: I0 R0 O0 C0 iterator_0 reducer_a_0 reducer_b_0 0 A Copy,, I1 R1 O1 C1 iterator_1 reducer_a_1 reducer_b_1 1 B Copy,, I2 R2 O2 C2 iterator_2 reducer_a_2 reducer_b_2 2 C Copy,, I3 R3 O3 C3 iterator_3 reducer_a_3 reducer_b_3 3 D Copy,,);
impl_tuple!(ReduceA5Factory ReduceA5 ReduceA5Async ReduceC5Factory ReduceC5 ReduceC5Async AsyncTuple5 Sum5 Copy: I0 R0 O0 C0 iterator_0 reducer_a_0 reducer_b_0 0 A Copy,, I1 R1 O1 C1 iterator_1 reducer_a_1 reducer_b_1 1 B Copy,, I2 R2 O2 C2 iterator_2 reducer_a_2 reducer_b_2 2 C Copy,, I3 R3 O3 C3 iterator_3 reducer_a_3 reducer_b_3 3 D Copy,, I4 R4 O4 C4 iterator_4 reducer_a_4 reducer_b_4 4 E Copy,,);
impl_tuple!(ReduceA6Factory ReduceA6 ReduceA6Async ReduceC6Factory ReduceC6 ReduceC6Async AsyncTuple6 Sum6 Copy: I0 R0 O0 C0 iterator_0 reducer_a_0 reducer_b_0 0 A Copy,, I1 R1 O1 C1 iterator_1 reducer_a_1 reducer_b_1 1 B Copy,, I2 R2 O2 C2 iterator_2 reducer_a_2 reducer_b_2 2 C Copy,, I3 R3 O3 C3 iterator_3 reducer_a_3 reducer_b_3 3 D Copy,, I4 R4 O4 C4 iterator_4 reducer_a_4 reducer_b_4 4 E Copy,, I5 R5 O5 C5 iterator_5 reducer_a_5 reducer_b_5 5 F Copy,,);
impl_tuple!(ReduceA7Factory ReduceA7 ReduceA7Async ReduceC7Factory ReduceC7 ReduceC7Async AsyncTuple7 Sum7 Copy: I0 R0 O0 C0 iterator_0 reducer_a_0 reducer_b_0 0 A Copy,, I1 R1 O1 C1 iterator_1 reducer_a_1 reducer_b_1 1 B Copy,, I2 R2 O2 C2 iterator_2 reducer_a_2 reducer_b_2 2 C Copy,, I3 R3 O3 C3 iterator_3 reducer_a_3 reducer_b_3 3 D Copy,, I4 R4 O4 C4 iterator_4 reducer_a_4 reducer_b_4 4 E Copy,, I5 R5 O5 C5 iterator_5 reducer_a_5 reducer_b_5 5 F Copy,, I6 R6 O6 C6 iterator_6 reducer_a_6 reducer_b_6 6 G Copy,,);
impl_tuple!(ReduceA8Factory ReduceA8 ReduceA8Async ReduceC8Factory ReduceC8 ReduceC8Async AsyncTuple8 Sum8 Copy: I0 R0 O0 C0 iterator_0 reducer_a_0 reducer_b_0 0 A Copy,, I1 R1 O1 C1 iterator_1 reducer_a_1 reducer_b_1 1 B Copy,, I2 R2 O2 C2 iterator_2 reducer_a_2 reducer_b_2 2 C Copy,, I3 R3 O3 C3 iterator_3 reducer_a_3 reducer_b_3 3 D Copy,, I4 R4 O4 C4 iterator_4 reducer_a_4 reducer_b_4 4 E Copy,, I5 R5 O5 C5 iterator_5 reducer_a_5 reducer_b_5 5 F Copy,, I6 R6 O6 C6 iterator_6 reducer_a_6 reducer_b_6 6 G Copy,, I7 R7 O7 C7 iterator_7 reducer_a_7 reducer_b_7 7 H Copy,,);

#[pin_project(project = PeekableProj)]
#[derive(Debug)]
#[must_use = "streams do nothing unless polled"]
pub struct Peekable<'a, St: Stream> {
	#[pin]
	stream: St,
	peeked: &'a mut Option<St::Item>,
}

impl<'a, St: Stream> Peekable<'a, St> {
	pub fn poll_peek(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<&mut St::Item>> {
		let PeekableProj { mut stream, peeked } = self.project();

		Poll::Ready(loop {
			if peeked.is_some() {
				break peeked.as_mut();
			} else if let Some(item) = ready!(stream.as_mut().poll_next(cx)) {
				**peeked = Some(item);
			} else {
				break None;
			}
		})
	}
}

impl<'a, S: Stream> Stream for Peekable<'a, S> {
	type Item = S::Item;

	fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
		let PeekableProj { stream, peeked } = self.project();
		if let Some(item) = peeked.take() {
			return Poll::Ready(Some(item));
		}
		stream.poll_next(cx)
	}

	fn size_hint(&self) -> (usize, Option<usize>) {
		let peek_len = if self.peeked.is_some() { 1 } else { 0 };
		let (lower, upper) = self.stream.size_hint();
		let lower = lower.saturating_add(peek_len);
		let upper = match upper {
			Some(x) => x.checked_add(peek_len),
			None => None,
		};
		(lower, upper)
	}
}