From d6d50f74e73ae1050acf87464ea49a162f183d3e Mon Sep 17 00:00:00 2001 From: David Gross Date: Thu, 5 Dec 2013 10:12:28 -0800 Subject: [PATCH 01/15] javadoc improvements (groupJoin, groupByUntil, timestamp w/scheduler) --- rxjava-core/src/main/java/rx/Observable.java | 61 +++++++++++++------- 1 file changed, 41 insertions(+), 20 deletions(-) diff --git a/rxjava-core/src/main/java/rx/Observable.java b/rxjava-core/src/main/java/rx/Observable.java index aa5f855290..9482a5f09f 100644 --- a/rxjava-core/src/main/java/rx/Observable.java +++ b/rxjava-core/src/main/java/rx/Observable.java @@ -2203,6 +2203,7 @@ public Observable throttleLast(long intervalDuration, TimeUnit unit, Schedule * @return an Observable that emits timestamped items from the source * Observable * @see RxJava Wiki: timestamp() + * @see MSDN: Observable.Timestamp */ public Observable> timestamp() { return create(OperationTimestamp.timestamp(this)); @@ -2211,11 +2212,14 @@ public Observable> timestamp() { /** * Wraps each item emitted by a source Observable in a {@link Timestamped} * object with timestamps provided by the given Scheduler. + *

+ * * * @param scheduler the {@link Scheduler} to use as a time source. * @return an Observable that emits timestamped items from the source * Observable with timestamps provided by the given Scheduler - * @see MSDN: Observable.Timestamp + * @see RxJava Wiki: timestamp() + * @see MSDN: Observable.Timestamp */ public Observable> timestamp(Scheduler scheduler) { return create(OperationTimestamp.timestamp(this, scheduler)); @@ -5115,6 +5119,8 @@ public Observable> groupBy(final Func1 + * * * @param right the other Observable to correlate values of this observable to * @param leftDuration function that returns an Observable which indicates the duration of @@ -5126,6 +5132,7 @@ public Observable> groupBy(final Func1RxJava Wiiki: groupJoin * @see MSDN: Observable.GroupJoin */ public Observable groupJoin(Observable right, Func1> leftDuration, @@ -6165,42 +6172,56 @@ public Observable>> toMultimap(Func1 * * * @param other the other Observable that has to emit an element before this * Observable's elements are relayed - * @return an Observable that skips elements from the source Observable until the secondary - * observable emits an element. + * @return an Observable that skips elements from the source Observable + * until the secondary Observable emits an element. * @see RxJava Wiki: skipUntil() - * @see MSDN: Observable.SkipUntil + * @see MSDN: Observable.SkipUntil */ public Observable skipUntil(Observable other) { return create(new OperationSkipUntil(this, other)); } - + /** - * Groups the elements of an observable sequence according to a specified key selector function until the duration observable expires for the key. - * @param keySelector A function to extract the key for each element. - * @param durationSelector A function to signal the expiration of a group. - * @return A sequence of observable groups, each of which corresponds to a unique key value, containing all elements that share that same key value. - * - * @see MSDN: Observable.GroupByUntil + * Groups the items emitted by an Observable according to a specified key + * selector function until the duration Observable expires for the key. + *

+ * + * + * @param keySelector a function to extract the key for each item + * @param durationSelector a function to signal the expiration of a group + * @return a sequence of Observable groups, each of which corresponds to a + * unique key value, containing all items that share that same + * key value + * @see RxJava Wiki: groupByUntil() + * @see MSDN: Observable.GroupByUntil */ public Observable> groupByUntil(Func1 keySelector, Func1, ? extends Observable> durationSelector) { return groupByUntil(keySelector, Functions.identity(), durationSelector); } /** - * Groups the elements of an observable sequence according to a specified key and value selector function until the duration observable expires for the key. - * @param keySelector A function to extract the key for each element. - * @param valueSelector A function to map each source element to an element in an onbservable group. - * @param durationSelector A function to signal the expiration of a group. - * @return A sequence of observable groups, each of which corresponds to a unique key value, containing all elements that share that same key value. - * - * @see MSDN: Observable.GroupByUntil + * Groups the items emitted by an Observable according to specified key and + * value selector functions until the duration Observable expires for the + * key. + *

+ * + * + * @param keySelector a function to extract the key for each item + * @param valueSelector a function to map each source element to an item + * emitted by an Observable group + * @param durationSelector a function to signal the expiration of a group + * @return a sequence of Observable groups, each of which corresponds to a + * unique key value, containing all items that share that same key + * value + * @see RxJava Wiki: groupByUntil() + * @see MSDN: Observable.GroupByUntil */ public Observable> groupByUntil(Func1 keySelector, Func1 valueSelector, Func1, ? extends Observable> durationSelector) { return create(new OperationGroupByUntil(this, keySelector, valueSelector, durationSelector)); From 1b31bca8c254a2d9186674d377f217d8fa6f00fb Mon Sep 17 00:00:00 2001 From: akarnokd Date: Fri, 6 Dec 2013 09:27:20 +0100 Subject: [PATCH 02/15] Operation Sample with Observable v2 --- rxjava-core/src/main/java/rx/Observable.java | 17 +- .../java/rx/operators/OperationSample.java | 89 ++++++++++ .../rx/operators/OperationSampleTest.java | 159 +++++++++++++++++- 3 files changed, 263 insertions(+), 2 deletions(-) diff --git a/rxjava-core/src/main/java/rx/Observable.java b/rxjava-core/src/main/java/rx/Observable.java index 9482a5f09f..7c12d21db3 100644 --- a/rxjava-core/src/main/java/rx/Observable.java +++ b/rxjava-core/src/main/java/rx/Observable.java @@ -4463,7 +4463,22 @@ public Observable sample(long period, TimeUnit unit) { public Observable sample(long period, TimeUnit unit, Scheduler scheduler) { return create(OperationSample.sample(this, period, unit, scheduler)); } - + + /** + * Return an Observable that emits the results of sampling the items + * emitted by this Observable when the sampler + * Observable produces an item or completes. + * + * @param sampler the Observable to use for sampling this + * + * @return an Observable that emits the results of sampling the items + * emitted by this Observable when the sampler + * Observable produces an item or completes. + */ + public Observable sample(Observable sampler) { + return create(new OperationSample.SampleWithObservable(this, sampler)); + } + /** * Returns an Observable that applies a function of your choosing to the * first item emitted by a source Observable, then feeds the result of that diff --git a/rxjava-core/src/main/java/rx/operators/OperationSample.java b/rxjava-core/src/main/java/rx/operators/OperationSample.java index 89da00e5fb..f5f8f96e3f 100644 --- a/rxjava-core/src/main/java/rx/operators/OperationSample.java +++ b/rxjava-core/src/main/java/rx/operators/OperationSample.java @@ -25,6 +25,8 @@ import rx.Scheduler; import rx.Subscription; import rx.concurrency.Schedulers; +import rx.subscriptions.CompositeSubscription; +import rx.subscriptions.SerialSubscription; import rx.subscriptions.Subscriptions; import rx.util.functions.Action0; @@ -115,4 +117,91 @@ public void call() { }); } } + /** + * Sample with the help of another observable. + * @see MSDN: Observable.Sample + */ + public static class SampleWithObservable implements OnSubscribeFunc { + final Observable source; + final Observable sampler; + public SampleWithObservable(Observable source, Observable sampler) { + this.source = source; + this.sampler = sampler; + } + @Override + public Subscription onSubscribe(Observer t1) { + return new ResultManager(t1).init(); + } + /** Observe source values. */ + class ResultManager implements Observer { + final Observer observer; + final CompositeSubscription cancel; + T value; + boolean valueTaken = true; + boolean done; + final Object guard; + public ResultManager(Observer observer) { + this.observer = observer; + cancel = new CompositeSubscription(); + guard = new Object(); + } + public Subscription init() { + cancel.add(source.subscribe(this)); + cancel.add(sampler.subscribe(new Sampler())); + + return cancel; + } + @Override + public void onNext(T args) { + synchronized (guard) { + valueTaken = false; + value = args; + } + } + + @Override + public void onError(Throwable e) { + synchronized (guard) { + if (!done) { + done = true; + observer.onError(e); + cancel.unsubscribe(); + } + } + } + + @Override + public void onCompleted() { + synchronized (guard) { + if (!done) { + done = true; + observer.onCompleted(); + cancel.unsubscribe(); + } + } + } + /** Take the latest value, but only once. */ + class Sampler implements Observer { + @Override + public void onNext(U args) { + synchronized (guard) { + if (!valueTaken && !done) { + valueTaken = true; + observer.onNext(value); + } + } + } + + @Override + public void onError(Throwable e) { + ResultManager.this.onError(e); + } + + @Override + public void onCompleted() { + ResultManager.this.onCompleted(); + } + } + } + } } diff --git a/rxjava-core/src/test/java/rx/operators/OperationSampleTest.java b/rxjava-core/src/test/java/rx/operators/OperationSampleTest.java index d868697f9f..18316889c1 100644 --- a/rxjava-core/src/test/java/rx/operators/OperationSampleTest.java +++ b/rxjava-core/src/test/java/rx/operators/OperationSampleTest.java @@ -15,7 +15,6 @@ */ package rx.operators; -import static org.mockito.Matchers.*; import static org.mockito.Mockito.*; import java.util.concurrent.TimeUnit; @@ -28,12 +27,14 @@ import rx.Observer; import rx.Subscription; import rx.concurrency.TestScheduler; +import rx.subjects.PublishSubject; import rx.subscriptions.Subscriptions; import rx.util.functions.Action0; public class OperationSampleTest { private TestScheduler scheduler; private Observer observer; + private Observer observer2; @Before @SuppressWarnings("unchecked") @@ -41,6 +42,7 @@ public class OperationSampleTest { public void before() { scheduler = new TestScheduler(); observer = mock(Observer.class); + observer2 = mock(Observer.class); } @Test @@ -105,4 +107,159 @@ public void call() { verify(observer, times(1)).onCompleted(); verify(observer, never()).onError(any(Throwable.class)); } + @Test + public void sampleWithSamplerNormal() { + PublishSubject source = PublishSubject.create(); + PublishSubject sampler = PublishSubject.create(); + + Observable m = source.sample(sampler); + m.subscribe(observer2); + + source.onNext(1); + source.onNext(2); + sampler.onNext(1); + source.onNext(3); + source.onNext(4); + sampler.onNext(2); + source.onCompleted(); + sampler.onNext(3); + + + InOrder inOrder = inOrder(observer2); + inOrder.verify(observer2, never()).onNext(1); + inOrder.verify(observer2, times(1)).onNext(2); + inOrder.verify(observer2, never()).onNext(3); + inOrder.verify(observer2, times(1)).onNext(4); + inOrder.verify(observer2, times(1)).onCompleted(); + verify(observer, never()).onError(any(Throwable.class)); + } + @Test + public void sampleWithSamplerNoDuplicates() { + PublishSubject source = PublishSubject.create(); + PublishSubject sampler = PublishSubject.create(); + + Observable m = source.sample(sampler); + m.subscribe(observer2); + + source.onNext(1); + source.onNext(2); + sampler.onNext(1); + sampler.onNext(1); + + source.onNext(3); + source.onNext(4); + sampler.onNext(2); + sampler.onNext(2); + + source.onCompleted(); + sampler.onNext(3); + + + InOrder inOrder = inOrder(observer2); + inOrder.verify(observer2, never()).onNext(1); + inOrder.verify(observer2, times(1)).onNext(2); + inOrder.verify(observer2, never()).onNext(3); + inOrder.verify(observer2, times(1)).onNext(4); + inOrder.verify(observer2, times(1)).onCompleted(); + verify(observer, never()).onError(any(Throwable.class)); + } + @Test + public void sampleWithSamplerTerminatingEarly() { + PublishSubject source = PublishSubject.create(); + PublishSubject sampler = PublishSubject.create(); + + Observable m = source.sample(sampler); + m.subscribe(observer2); + + source.onNext(1); + source.onNext(2); + sampler.onNext(1); + sampler.onCompleted(); + + source.onNext(3); + source.onNext(4); + + + + InOrder inOrder = inOrder(observer2); + inOrder.verify(observer2, never()).onNext(1); + inOrder.verify(observer2, times(1)).onNext(2); + inOrder.verify(observer2, times(1)).onCompleted(); + inOrder.verify(observer2, never()).onNext(any()); + verify(observer, never()).onError(any(Throwable.class)); + } + @Test + public void sampleWithSamplerEmitAndTerminate() { + PublishSubject source = PublishSubject.create(); + PublishSubject sampler = PublishSubject.create(); + + Observable m = source.sample(sampler); + m.subscribe(observer2); + + source.onNext(1); + source.onNext(2); + sampler.onNext(1); + source.onNext(3); + source.onCompleted(); + sampler.onNext(2); + sampler.onCompleted(); + + InOrder inOrder = inOrder(observer2); + inOrder.verify(observer2, never()).onNext(1); + inOrder.verify(observer2, times(1)).onNext(2); + inOrder.verify(observer2, never()).onNext(3); + inOrder.verify(observer2, times(1)).onCompleted(); + inOrder.verify(observer2, never()).onNext(any()); + verify(observer, never()).onError(any(Throwable.class)); + } + @Test + public void sampleWithSamplerEmptySource() { + PublishSubject source = PublishSubject.create(); + PublishSubject sampler = PublishSubject.create(); + + Observable m = source.sample(sampler); + m.subscribe(observer2); + + source.onCompleted(); + sampler.onNext(1); + + InOrder inOrder = inOrder(observer2); + inOrder.verify(observer2, times(1)).onCompleted(); + verify(observer2, never()).onNext(any()); + verify(observer, never()).onError(any(Throwable.class)); + } + @Test + public void sampleWithSamplerSourceThrows() { + PublishSubject source = PublishSubject.create(); + PublishSubject sampler = PublishSubject.create(); + + Observable m = source.sample(sampler); + m.subscribe(observer2); + + source.onNext(1); + source.onError(new RuntimeException("Forced failure!")); + sampler.onNext(1); + + InOrder inOrder = inOrder(observer2); + inOrder.verify(observer2, times(1)).onError(any(Throwable.class)); + verify(observer2, never()).onNext(any()); + verify(observer, never()).onCompleted(); + } + @Test + public void sampleWithSamplerThrows() { + PublishSubject source = PublishSubject.create(); + PublishSubject sampler = PublishSubject.create(); + + Observable m = source.sample(sampler); + m.subscribe(observer2); + + source.onNext(1); + sampler.onNext(1); + sampler.onError(new RuntimeException("Forced failure!")); + + InOrder inOrder = inOrder(observer2); + inOrder.verify(observer2, times(1)).onNext(1); + inOrder.verify(observer2, times(1)).onError(any(RuntimeException.class)); + verify(observer, never()).onCompleted(); + } } From dab56f7989b7f42c6f0c047e253d0748493dc027 Mon Sep 17 00:00:00 2001 From: akarnokd Date: Fri, 6 Dec 2013 09:51:25 +0100 Subject: [PATCH 03/15] ObserveOn fix for observing the same source on the same scheduler by two different observers. --- .../java/rx/operators/OperationObserveOn.java | 97 +++++++++---------- .../rx/operators/OperationObserveOnTest.java | 74 ++++++++++++++ 2 files changed, 119 insertions(+), 52 deletions(-) diff --git a/rxjava-core/src/main/java/rx/operators/OperationObserveOn.java b/rxjava-core/src/main/java/rx/operators/OperationObserveOn.java index fbfda6ef1b..22876a4ced 100644 --- a/rxjava-core/src/main/java/rx/operators/OperationObserveOn.java +++ b/rxjava-core/src/main/java/rx/operators/OperationObserveOn.java @@ -27,6 +27,7 @@ import rx.concurrency.CurrentThreadScheduler; import rx.concurrency.ImmediateScheduler; import rx.subscriptions.CompositeSubscription; +import rx.subscriptions.SerialSubscription; import rx.subscriptions.Subscriptions; import rx.util.functions.Action0; import rx.util.functions.Action1; @@ -46,10 +47,6 @@ public static OnSubscribeFunc observeOn(Observable source, S private static class ObserveOn implements OnSubscribeFunc { private final Observable source; private final Scheduler scheduler; - private volatile Scheduler recursiveScheduler; - - final ConcurrentLinkedQueue> queue = new ConcurrentLinkedQueue>(); - final AtomicInteger counter = new AtomicInteger(0); public ObserveOn(Observable source, Scheduler scheduler) { this.source = source; @@ -65,71 +62,67 @@ public Subscription onSubscribe(final Observer observer) { // do nothing if we request CurrentThreadScheduler so we don't invoke overhead return source.subscribe(observer); } else { - return observeOn(observer, scheduler); + return new Observation(observer).init(); } } + /** Observe through individual queue per observer. */ + private class Observation implements Action1> { + final Observer observer; + final CompositeSubscription s; + final ConcurrentLinkedQueue> queue; + final AtomicInteger counter; + private volatile Scheduler recursiveScheduler; + public Observation(Observer observer) { + this.observer = observer; + this.queue = new ConcurrentLinkedQueue>(); + this.counter = new AtomicInteger(0); + this.s = new CompositeSubscription(); + } + public Subscription init() { + s.add(source.materialize().subscribe(this)); + return s; + } - public Subscription observeOn(final Observer observer, final Scheduler scheduler) { - final CompositeSubscription s = new CompositeSubscription(); - - s.add(source.materialize().subscribe(new Action1>() { - - @Override - public void call(Notification e) { - // this must happen before 'counter' is used to provide synchronization between threads - queue.offer(e); - - // we now use counter to atomically determine if we need to start processing or not - // it will be 0 if it's the first notification or the scheduler has finished processing work - // and we need to start doing it again - if (counter.getAndIncrement() == 0) { - if (recursiveScheduler == null) { - s.add(scheduler.schedule(null, new Func2() { - + @Override + public void call(Notification e) { + queue.offer(e); + if (counter.getAndIncrement() == 0) { + if (recursiveScheduler == null) { + s.add(scheduler.schedule(null, new Func2() { @Override public Subscription call(Scheduler innerScheduler, T state) { // record innerScheduler so 'processQueue' can use it for all subsequent executions recursiveScheduler = innerScheduler; - processQueue(s, observer); + processQueue(); return Subscriptions.empty(); } })); - } else { - processQueue(s, observer); - } + } else { + processQueue(); } - } - })); - - return s; - } - - /** - * This uses 'recursiveScheduler' NOT 'scheduler' as it should reuse the same scheduler each time it processes. - * This means it must first get the recursiveScheduler when it first executes. - */ - private void processQueue(final CompositeSubscription s, final Observer observer) { + } + void processQueue() { + s.add(recursiveScheduler.schedule(new Action1() { + @Override + public void call(Action0 self) { + Notification not = queue.poll(); + if (not != null) { + not.accept(observer); + } - s.add(recursiveScheduler.schedule(new Action1() { - @Override - public void call(Action0 self) { - Notification not = queue.poll(); - if (not != null) { - not.accept(observer); - } + // decrement count and if we still have work to do + // recursively schedule ourselves to process again + if (counter.decrementAndGet() > 0) { + self.call(); + } - // decrement count and if we still have work to do - // recursively schedule ourselves to process again - if (counter.decrementAndGet() > 0) { - self.call(); } - - } - })); + })); + } } } -} +} \ No newline at end of file diff --git a/rxjava-core/src/test/java/rx/operators/OperationObserveOnTest.java b/rxjava-core/src/test/java/rx/operators/OperationObserveOnTest.java index 15f1a44fb3..63fa0e4f27 100644 --- a/rxjava-core/src/test/java/rx/operators/OperationObserveOnTest.java +++ b/rxjava-core/src/test/java/rx/operators/OperationObserveOnTest.java @@ -31,6 +31,7 @@ import rx.Observable; import rx.Observer; import rx.concurrency.Schedulers; +import rx.concurrency.TestScheduler; import rx.util.functions.Action1; public class OperationObserveOnTest { @@ -132,4 +133,77 @@ public void call(String t1) { inOrder.verify(observer, times(1)).onCompleted(); } + @Test + public void observeOnTheSameSchedulerTwice() { + TestScheduler scheduler = new TestScheduler(); + + Observable o = Observable.from(1, 2, 3); + Observable o2 = o.observeOn(scheduler); + + @SuppressWarnings("unchecked") + Observer observer1 = mock(Observer.class); + @SuppressWarnings("unchecked") + Observer observer2 = mock(Observer.class); + + InOrder inOrder1 = inOrder(observer1); + InOrder inOrder2 = inOrder(observer2); + + o2.subscribe(observer1); + o2.subscribe(observer2); + + scheduler.advanceTimeBy(1, TimeUnit.SECONDS); + + inOrder1.verify(observer1, times(1)).onNext(1); + inOrder1.verify(observer1, times(1)).onNext(2); + inOrder1.verify(observer1, times(1)).onNext(3); + inOrder1.verify(observer1, times(1)).onCompleted(); + verify(observer1, never()).onError(any(Throwable.class)); + inOrder1.verifyNoMoreInteractions(); + + inOrder2.verify(observer2, times(1)).onNext(1); + inOrder2.verify(observer2, times(1)).onNext(2); + inOrder2.verify(observer2, times(1)).onNext(3); + inOrder2.verify(observer2, times(1)).onCompleted(); + verify(observer2, never()).onError(any(Throwable.class)); + inOrder2.verifyNoMoreInteractions(); + + } + @Test + public void observeSameOnMultipleSchedulers() { + TestScheduler scheduler1 = new TestScheduler(); + TestScheduler scheduler2 = new TestScheduler(); + + Observable o = Observable.from(1, 2, 3); + Observable o1 = o.observeOn(scheduler1); + Observable o2 = o.observeOn(scheduler2); + + @SuppressWarnings("unchecked") + Observer observer1 = mock(Observer.class); + @SuppressWarnings("unchecked") + Observer observer2 = mock(Observer.class); + + InOrder inOrder1 = inOrder(observer1); + InOrder inOrder2 = inOrder(observer2); + + o1.subscribe(observer1); + o2.subscribe(observer2); + + scheduler1.advanceTimeBy(1, TimeUnit.SECONDS); + scheduler2.advanceTimeBy(1, TimeUnit.SECONDS); + + inOrder1.verify(observer1, times(1)).onNext(1); + inOrder1.verify(observer1, times(1)).onNext(2); + inOrder1.verify(observer1, times(1)).onNext(3); + inOrder1.verify(observer1, times(1)).onCompleted(); + verify(observer1, never()).onError(any(Throwable.class)); + inOrder1.verifyNoMoreInteractions(); + + inOrder2.verify(observer2, times(1)).onNext(1); + inOrder2.verify(observer2, times(1)).onNext(2); + inOrder2.verify(observer2, times(1)).onNext(3); + inOrder2.verify(observer2, times(1)).onCompleted(); + verify(observer2, never()).onError(any(Throwable.class)); + inOrder2.verifyNoMoreInteractions(); + + } } From 7729df4cf7958d87aede1e7cca57cb877381f1d8 Mon Sep 17 00:00:00 2001 From: akarnokd Date: Fri, 6 Dec 2013 11:30:13 +0100 Subject: [PATCH 04/15] Removed Opening and Closing historical artifacts. --- .../main/scala/rx/lang/scala/Observable.scala | 47 ++++++++-------- .../scala/rx/lang/scala/util/package.scala | 24 -------- rxjava-core/src/main/java/rx/Observable.java | 56 +++++++++---------- .../java/rx/operators/ChunkedOperation.java | 28 +++++----- .../java/rx/operators/OperationBuffer.java | 18 +++--- .../java/rx/operators/OperationWindow.java | 18 +++--- .../src/main/java/rx/util/Closing.java | 20 ------- .../src/main/java/rx/util/Closings.java | 28 ---------- .../src/main/java/rx/util/Opening.java | 20 ------- .../src/main/java/rx/util/Openings.java | 28 ---------- .../rx/operators/OperationBufferTest.java | 32 +++++------ .../rx/operators/OperationWindowTest.java | 32 +++++------ 12 files changed, 107 insertions(+), 244 deletions(-) delete mode 100644 rxjava-core/src/main/java/rx/util/Closing.java delete mode 100644 rxjava-core/src/main/java/rx/util/Closings.java delete mode 100644 rxjava-core/src/main/java/rx/util/Opening.java delete mode 100644 rxjava-core/src/main/java/rx/util/Openings.java diff --git a/language-adaptors/rxjava-scala/src/main/scala/rx/lang/scala/Observable.scala b/language-adaptors/rxjava-scala/src/main/scala/rx/lang/scala/Observable.scala index 9999ed9751..b190f23aff 100644 --- a/language-adaptors/rxjava-scala/src/main/scala/rx/lang/scala/Observable.scala +++ b/language-adaptors/rxjava-scala/src/main/scala/rx/lang/scala/Observable.scala @@ -302,45 +302,44 @@ trait Observable[+T] * Creates an Observable which produces buffers of collected values. * * This Observable produces connected non-overlapping buffers. The current buffer is - * emitted and replaced with a new buffer when the Observable produced by the specified function produces a [[rx.lang.scala.util.Closing]] object. The function will then + * emitted and replaced with a new buffer when the Observable produced by the specified function produces an object. The function will then * be used to create a new Observable to listen for the end of the next buffer. * * @param closings * The function which is used to produce an [[rx.lang.scala.Observable]] for every buffer created. - * When this [[rx.lang.scala.Observable]] produces a [[rx.lang.scala.util.Closing]] object, the associated buffer + * When this [[rx.lang.scala.Observable]] produces an object, the associated buffer * is emitted and replaced with a new one. * @return * An [[rx.lang.scala.Observable]] which produces connected non-overlapping buffers, which are emitted - * when the current [[rx.lang.scala.Observable]] created with the function argument produces a [[rx.lang.scala.util.Closing]] object. + * when the current [[rx.lang.scala.Observable]] created with the function argument produces an object. */ - def buffer(closings: () => Observable[Closing]) : Observable[Seq[T]] = { + def buffer[Closing](closings: () => Observable[_ <: Closing]) : Observable[Seq[T]] = { val f: Func0[_ <: rx.Observable[_ <: Closing]] = closings().asJavaObservable - val jObs: rx.Observable[_ <: java.util.List[_]] = asJavaObservable.buffer(f) + val jObs: rx.Observable[_ <: java.util.List[_]] = asJavaObservable.buffer[Closing](f) Observable.jObsOfListToScObsOfSeq(jObs.asInstanceOf[rx.Observable[_ <: java.util.List[T]]]) } - /** * Creates an Observable which produces buffers of collected values. * * This Observable produces buffers. Buffers are created when the specified `openings` - * Observable produces a [[rx.lang.scala.util.Opening]] object. Additionally the function argument + * Observable produces an object. Additionally the function argument * is used to create an Observable which produces [[rx.lang.scala.util.Closing]] objects. When this * Observable produces such an object, the associated buffer is emitted. * * @param openings - * The [[rx.lang.scala.Observable]] which, when it produces a [[rx.lang.scala.util.Opening]] object, will cause + * The [[rx.lang.scala.Observable]] which, when it produces an object, will cause * another buffer to be created. * @param closings * The function which is used to produce an [[rx.lang.scala.Observable]] for every buffer created. - * When this [[rx.lang.scala.Observable]] produces a [[rx.lang.scala.util.Closing]] object, the associated buffer + * When this [[rx.lang.scala.Observable]] produces an object, the associated buffer * is emitted. * @return * An [[rx.lang.scala.Observable]] which produces buffers which are created and emitted when the specified [[rx.lang.scala.Observable]]s publish certain objects. */ - def buffer(openings: Observable[Opening], closings: Opening => Observable[Closing]): Observable[Seq[T]] = { + def buffer[Opening, Closing](openings: Observable[Opening], closings: Opening => Observable[Closing]): Observable[Seq[T]] = { val opening: rx.Observable[_ <: Opening] = openings.asJavaObservable - val closing: Func1[Opening, _ <: rx.Observable[_ <: Closing]] = (o: Opening) => closings(o).asJavaObservable - val jObs: rx.Observable[_ <: java.util.List[_]] = asJavaObservable.buffer(opening, closing) + val closing: Func1[_ >: Opening, _ <: rx.Observable[_ <: Closing]] = (o: Opening) => closings(o).asJavaObservable + val jObs: rx.Observable[_ <: java.util.List[_]] = asJavaObservable.buffer[Opening, Closing](opening, closing) Observable.jObsOfListToScObsOfSeq(jObs.asInstanceOf[rx.Observable[_ <: java.util.List[T]]]) } @@ -512,47 +511,47 @@ trait Observable[+T] /** * Creates an Observable which produces windows of collected values. This Observable produces connected * non-overlapping windows. The current window is emitted and replaced with a new window when the - * Observable produced by the specified function produces a [[rx.lang.scala.util.Closing]] object. + * Observable produced by the specified function produces an object. * The function will then be used to create a new Observable to listen for the end of the next * window. * * @param closings * The function which is used to produce an [[rx.lang.scala.Observable]] for every window created. - * When this [[rx.lang.scala.Observable]] produces a [[rx.lang.scala.util.Closing]] object, the associated window + * When this [[rx.lang.scala.Observable]] produces an object, the associated window * is emitted and replaced with a new one. * @return * An [[rx.lang.scala.Observable]] which produces connected non-overlapping windows, which are emitted - * when the current [[rx.lang.scala.Observable]] created with the function argument produces a [[rx.lang.scala.util.Closing]] object. + * when the current [[rx.lang.scala.Observable]] created with the function argument produces an object. */ - def window(closings: () => Observable[Closing]): Observable[Observable[T]] = { + def window[Closing](closings: () => Observable[Closing]): Observable[Observable[T]] = { val func : Func0[_ <: rx.Observable[_ <: Closing]] = closings().asJavaObservable - val o1: rx.Observable[_ <: rx.Observable[_]] = asJavaObservable.window(func) - val o2 = toScalaObservable[rx.Observable[_]](o1).map((x: rx.Observable[_]) => { + val o1: rx.Observable[_ <: rx.Observable[_]] = asJavaObservable.window[Closing](func) + val o2 = Observable[rx.Observable[_]](o1).map((x: rx.Observable[_]) => { val x2 = x.asInstanceOf[rx.Observable[_ <: T]] - toScalaObservable[T](x2) + Observable[T](x2) }) o2 } /** * Creates an Observable which produces windows of collected values. This Observable produces windows. - * Chunks are created when the specified `openings` Observable produces a [[rx.lang.scala.util.Opening]] object. + * Chunks are created when the specified `openings` Observable produces an object. * Additionally the `closings` argument is used to create an Observable which produces [[rx.lang.scala.util.Closing]] objects. * When this Observable produces such an object, the associated window is emitted. * * @param openings - * The [[rx.lang.scala.Observable]] which when it produces a [[rx.lang.scala.util.Opening]] object, will cause + * The [[rx.lang.scala.Observable]] which when it produces an object, will cause * another window to be created. * @param closings * The function which is used to produce an [[rx.lang.scala.Observable]] for every window created. - * When this [[rx.lang.scala.Observable]] produces a [[rx.lang.scala.util.Closing]] object, the associated window + * When this [[rx.lang.scala.Observable]] produces an object, the associated window * is emitted. * @return * An [[rx.lang.scala.Observable]] which produces windows which are created and emitted when the specified [[rx.lang.scala.Observable]]s publish certain objects. */ - def window(openings: Observable[Opening], closings: Opening => Observable[Closing]) = { + def window[Opening, Closing](openings: Observable[Opening], closings: Opening => Observable[Closing]) = { Observable.jObsOfJObsToScObsOfScObs( - asJavaObservable.window(openings.asJavaObservable, (op: Opening) => closings(op).asJavaObservable)) + asJavaObservable.window[Opening, Closing](openings.asJavaObservable, (op: Opening) => closings(op).asJavaObservable)) : Observable[Observable[T]] // SI-7818 } diff --git a/language-adaptors/rxjava-scala/src/main/scala/rx/lang/scala/util/package.scala b/language-adaptors/rxjava-scala/src/main/scala/rx/lang/scala/util/package.scala index ed19d849ab..45afef4a4d 100644 --- a/language-adaptors/rxjava-scala/src/main/scala/rx/lang/scala/util/package.scala +++ b/language-adaptors/rxjava-scala/src/main/scala/rx/lang/scala/util/package.scala @@ -20,30 +20,6 @@ package rx.lang.scala */ package object util { - /** - * Tagging interface for objects which can open buffers. - * @see [[Observable `Observable.buffer(Observable[Opening], Opening => Observable[Closing])`]] - */ - type Opening = rx.util.Opening - - /** - * Creates an object which can open buffers. - * @see [[Observable `Observable.buffer(Observable[Opening], Opening => Observable[Closing])`]] - */ - def Opening() = rx.util.Openings.create() - - /** - * Tagging interface for objects which can close buffers. - * @see [[Observable `Observable.buffer(Observable[Opening], Opening => Observable[Closing])`]] - */ - type Closing = rx.util.Closing - - /** - * Creates an object which can close buffers. - * @see [[Observable `Observable.buffer(Observable[Opening], Opening => Observable[Closing])`]] - */ - def Closing() = rx.util.Closings.create() - // rx.util.Range not needed because there's a standard Scala Range } diff --git a/rxjava-core/src/main/java/rx/Observable.java b/rxjava-core/src/main/java/rx/Observable.java index 9482a5f09f..c682e1bc00 100644 --- a/rxjava-core/src/main/java/rx/Observable.java +++ b/rxjava-core/src/main/java/rx/Observable.java @@ -111,9 +111,7 @@ import rx.subjects.ReplaySubject; import rx.subjects.Subject; import rx.subscriptions.Subscriptions; -import rx.util.Closing; import rx.util.OnErrorNotImplementedException; -import rx.util.Opening; import rx.util.Range; import rx.util.TimeInterval; import rx.util.Timestamped; @@ -2812,31 +2810,31 @@ public static Observable combineLates return create(OperationCombineLatest.combineLatest(o1, o2, o3, o4, o5, o6, o7, o8, o9, combineFunction)); } - /** +/** * Creates an Observable that produces buffers of collected items. *

* *

* This Observable produces connected, non-overlapping buffers. The current * buffer is emitted and replaced with a new buffer when the Observable - * produced by the specified bufferClosingSelector produces a - * {@link rx.util.Closing} object. The bufferClosingSelector + * produced by the specified bufferClosingSelector produces an + * object. The bufferClosingSelector * will then be used to create a new Observable to listen for the end of * the next buffer. * * @param bufferClosingSelector the {@link Func0} which is used to produce * an {@link Observable} for every buffer * created. When this {@link Observable} - * produces a {@link rx.util.Closing} object, + * produces an object, * the associated buffer is emitted and * replaced with a new one. * @return an {@link Observable} which produces connected, non-overlapping * buffers, which are emitted when the current {@link Observable} - * created with the {@link Func0} argument produces a - * {@link rx.util.Closing} object + * created with the {@link Func0} argument produces an + * object * @see RxJava Wiki: buffer() */ - public Observable> buffer(Func0> bufferClosingSelector) { + public Observable> buffer(Func0> bufferClosingSelector) { return create(OperationBuffer.buffer(this, bufferClosingSelector)); } @@ -2846,26 +2844,26 @@ public Observable> buffer(Func0> * *

* This Observable produces buffers. Buffers are created when the specified - * bufferOpenings Observable produces a {@link rx.util.Opening} + * bufferOpenings Observable produces an * object. Additionally the bufferClosingSelector argument is - * used to create an Observable which produces {@link rx.util.Closing} + * used to create an Observable which produces * objects. When this Observable produces such an object, the associated * buffer is emitted. * - * @param bufferOpenings the {@link Observable} that, when it produces a - * {@link rx.util.Opening} object, will cause another + * @param bufferOpenings the {@link Observable} that, when it produces an + * object, will cause another * buffer to be created * @param bufferClosingSelector the {@link Func1} that is used to produce * an {@link Observable} for every buffer * created. When this {@link Observable} - * produces a {@link rx.util.Closing} object, + * produces an object, * the associated buffer is emitted. * @return an {@link Observable} that produces buffers that are created and * emitted when the specified {@link Observable}s publish certain * objects * @see RxJava Wiki: buffer() */ - public Observable> buffer(Observable bufferOpenings, Func1> bufferClosingSelector) { + public Observable> buffer(Observable bufferOpenings, Func1> bufferClosingSelector) { return create(OperationBuffer.buffer(this, bufferOpenings, bufferClosingSelector)); } @@ -3062,8 +3060,8 @@ public Observable> buffer(long timespan, long timeshift, TimeUnit unit, * Creates an Observable that produces windows of collected items. This * Observable produces connected, non-overlapping windows. The current * window is emitted and replaced with a new window when the Observable - * produced by the specified closingSelector produces a - * {@link rx.util.Closing} object. The closingSelector will + * produced by the specified closingSelector produces an + * object. The closingSelector will * then be used to create a new Observable to listen for the end of the next * window. *

@@ -3071,45 +3069,45 @@ public Observable> buffer(long timespan, long timeshift, TimeUnit unit, * * @param closingSelector the {@link Func0} used to produce an * {@link Observable} for every window created. When this - * {@link Observable} emits a {@link rx.util.Closing} object, the + * {@link Observable} emits an object, the * associated window is emitted and replaced with a new one. * @return an {@link Observable} that produces connected, non-overlapping * windows, which are emitted when the current {@link Observable} - * created with the closingSelector argument emits a - * {@link rx.util.Closing} object. + * created with the closingSelector argument emits an + * object. * @see RxJava Wiki: window() */ - public Observable> window(Func0> closingSelector) { + public Observable> window(Func0> closingSelector) { return create(OperationWindow.window(this, closingSelector)); } /** * Creates an Observable that produces windows of collected items. This * Observable produces windows. Chunks are created when the - * windowOpenings Observable produces a {@link rx.util.Opening} + * windowOpenings Observable produces an * object. Additionally the closingSelector argument creates an - * Observable that produces {@link rx.util.Closing} objects. When this + * Observable that produces objects. When this * Observable produces such an object, the associated window is emitted. *

* * - * @param windowOpenings the {@link Observable} that, when it produces a - * {@link rx.util.Opening} object, causes another + * @param windowOpenings the {@link Observable} that, when it produces an + * object, causes another * window to be created * @param closingSelector the {@link Func1} that produces an * {@link Observable} for every window created. When - * this {@link Observable} produces a - * {@link rx.util.Closing} object, the associated + * this {@link Observable} produces an + * object, the associated * window is emitted. * @return an {@link Observable} that produces windows that are created and * emitted when the specified {@link Observable}s publish certain * objects * @see RxJava Wiki: window() */ - public Observable> window(Observable windowOpenings, Func1> closingSelector) { + public Observable> window(Observable windowOpenings, Func1> closingSelector) { return create(OperationWindow.window(this, windowOpenings, closingSelector)); } - + /** * Creates an Observable that produces windows of collected items. This * Observable produces connected, non-overlapping windows, each containing diff --git a/rxjava-core/src/main/java/rx/operators/ChunkedOperation.java b/rxjava-core/src/main/java/rx/operators/ChunkedOperation.java index cabc1309e6..1d3b762891 100644 --- a/rxjava-core/src/main/java/rx/operators/ChunkedOperation.java +++ b/rxjava-core/src/main/java/rx/operators/ChunkedOperation.java @@ -28,8 +28,6 @@ import rx.Observer; import rx.Scheduler; import rx.Subscription; -import rx.util.Closing; -import rx.util.Opening; import rx.util.functions.Action0; import rx.util.functions.Action1; import rx.util.functions.Func0; @@ -449,13 +447,13 @@ public void stop() { * The type of object all internal {@link rx.operators.ChunkedOperation.Chunk} objects record. * The type of object being tracked by the {@link Chunk} */ - protected static class ObservableBasedSingleChunkCreator implements ChunkCreator { + protected static class ObservableBasedSingleChunkCreator implements ChunkCreator { private final SafeObservableSubscription subscription = new SafeObservableSubscription(); - private final Func0> chunkClosingSelector; + private final Func0> chunkClosingSelector; private final NonOverlappingChunks chunks; - public ObservableBasedSingleChunkCreator(NonOverlappingChunks chunks, Func0> chunkClosingSelector) { + public ObservableBasedSingleChunkCreator(NonOverlappingChunks chunks, Func0> chunkClosingSelector) { this.chunks = chunks; this.chunkClosingSelector = chunkClosingSelector; @@ -464,10 +462,10 @@ public ObservableBasedSingleChunkCreator(NonOverlappingChunks chunks, Func } private void listenForChunkEnd() { - Observable closingObservable = chunkClosingSelector.call(); - closingObservable.subscribe(new Action1() { + Observable closingObservable = chunkClosingSelector.call(); + closingObservable.subscribe(new Action1() { @Override - public void call(Closing closing) { + public void call(TClosing closing) { chunks.emitAndReplaceChunk(); listenForChunkEnd(); } @@ -495,20 +493,20 @@ public void stop() { * The type of object all internal {@link rx.operators.ChunkedOperation.Chunk} objects record. * The type of object being tracked by the {@link Chunk} */ - protected static class ObservableBasedMultiChunkCreator implements ChunkCreator { + protected static class ObservableBasedMultiChunkCreator implements ChunkCreator { private final SafeObservableSubscription subscription = new SafeObservableSubscription(); - public ObservableBasedMultiChunkCreator(final OverlappingChunks chunks, Observable openings, final Func1> chunkClosingSelector) { - subscription.wrap(openings.subscribe(new Action1() { + public ObservableBasedMultiChunkCreator(final OverlappingChunks chunks, Observable openings, final Func1> chunkClosingSelector) { + subscription.wrap(openings.subscribe(new Action1() { @Override - public void call(Opening opening) { + public void call(TOpening opening) { final Chunk chunk = chunks.createChunk(); - Observable closingObservable = chunkClosingSelector.call(opening); + Observable closingObservable = chunkClosingSelector.call(opening); - closingObservable.subscribe(new Action1() { + closingObservable.subscribe(new Action1() { @Override - public void call(Closing closing) { + public void call(TClosing closing) { chunks.emitChunk(chunk); } }); diff --git a/rxjava-core/src/main/java/rx/operators/OperationBuffer.java b/rxjava-core/src/main/java/rx/operators/OperationBuffer.java index c9aba14411..02d0ce4573 100644 --- a/rxjava-core/src/main/java/rx/operators/OperationBuffer.java +++ b/rxjava-core/src/main/java/rx/operators/OperationBuffer.java @@ -24,8 +24,6 @@ import rx.Scheduler; import rx.Subscription; import rx.concurrency.Schedulers; -import rx.util.Closing; -import rx.util.Opening; import rx.util.functions.Func0; import rx.util.functions.Func1; @@ -42,7 +40,7 @@ public Buffer call() { /** *

This method creates a {@link Func1} object which represents the buffer operation. This operation takes - * values from the specified {@link Observable} source and stores them in a buffer until the {@link Observable} constructed using the {@link Func0} argument, produces a {@link rx.util.Closing} + * values from the specified {@link Observable} source and stores them in a buffer until the {@link Observable} constructed using the {@link Func0} argument, produces a * value. The buffer is then * emitted, and a new buffer is created to replace it. A new {@link Observable} will be constructed using the * provided {@link Func0} object, which will determine when this new buffer is emitted. When the source {@link Observable} completes or produces an error, the current buffer is emitted, and the @@ -56,17 +54,17 @@ public Buffer call() { * The {@link Observable} which produces values. * @param bufferClosingSelector * A {@link Func0} object which produces {@link Observable}s. These {@link Observable}s determine when a buffer is emitted and replaced by simply - * producing an {@link rx.util.Closing} object. + * producing an object. * @return * the {@link Func1} object representing the specified buffer operation. */ - public static OnSubscribeFunc> buffer(final Observable source, final Func0> bufferClosingSelector) { + public static OnSubscribeFunc> buffer(final Observable source, final Func0> bufferClosingSelector) { return new OnSubscribeFunc>() { @Override public Subscription onSubscribe(Observer> observer) { NonOverlappingChunks> buffers = new NonOverlappingChunks>(observer, OperationBuffer. bufferMaker()); - ChunkCreator creator = new ObservableBasedSingleChunkCreator>(buffers, bufferClosingSelector); + ChunkCreator creator = new ObservableBasedSingleChunkCreator, TClosing>(buffers, bufferClosingSelector); return source.subscribe(new ChunkObserver>(buffers, observer, creator)); } }; @@ -79,7 +77,7 @@ public Subscription onSubscribe(Observer> observer) { * *

Chunks can be created by pushing a {@link rx.util.Opening} value to the "bufferOpenings" {@link Observable}. * This creates a new buffer which will then start recording values which are produced by the "source" {@link Observable}. Additionally the "bufferClosingSelector" will be used to construct an - * {@link Observable} which can produce {@link rx.util.Closing} values. When it does so it will close this (and only this) newly created + * {@link Observable} which can produce values. When it does so it will close this (and only this) newly created * buffer. When the source {@link Observable} completes or produces an error, all chunks are emitted, and the * event is propagated to all subscribed {@link Observer}s.

* @@ -93,16 +91,16 @@ public Subscription onSubscribe(Observer> observer) { * create a new buffer which instantly starts recording the "source" {@link Observable}. * @param bufferClosingSelector * A {@link Func0} object which produces {@link Observable}s. These {@link Observable}s determine when a buffer is emitted and replaced by simply - * producing an {@link rx.util.Closing} object. + * producing an object. * @return * the {@link Func1} object representing the specified buffer operation. */ - public static OnSubscribeFunc> buffer(final Observable source, final Observable bufferOpenings, final Func1> bufferClosingSelector) { + public static OnSubscribeFunc> buffer(final Observable source, final Observable bufferOpenings, final Func1> bufferClosingSelector) { return new OnSubscribeFunc>() { @Override public Subscription onSubscribe(final Observer> observer) { OverlappingChunks> buffers = new OverlappingChunks>(observer, OperationBuffer. bufferMaker()); - ChunkCreator creator = new ObservableBasedMultiChunkCreator>(buffers, bufferOpenings, bufferClosingSelector); + ChunkCreator creator = new ObservableBasedMultiChunkCreator, TOpening, TClosing>(buffers, bufferOpenings, bufferClosingSelector); return source.subscribe(new ChunkObserver>(buffers, observer, creator)); } }; diff --git a/rxjava-core/src/main/java/rx/operators/OperationWindow.java b/rxjava-core/src/main/java/rx/operators/OperationWindow.java index d7ff20e061..f18a700bd2 100644 --- a/rxjava-core/src/main/java/rx/operators/OperationWindow.java +++ b/rxjava-core/src/main/java/rx/operators/OperationWindow.java @@ -23,8 +23,6 @@ import rx.Scheduler; import rx.Subscription; import rx.concurrency.Schedulers; -import rx.util.Closing; -import rx.util.Opening; import rx.util.functions.Func0; import rx.util.functions.Func1; @@ -42,7 +40,7 @@ public Window call() { /** *

This method creates a {@link rx.util.functions.Func1} object which represents the window operation. This operation takes * values from the specified {@link rx.Observable} source and stores them in a window until the {@link rx.Observable} constructed using the {@link rx.util.functions.Func0} argument, produces a - * {@link rx.util.Closing} value. The window is then + * value. The window is then * emitted, and a new window is created to replace it. A new {@link rx.Observable} will be constructed using the * provided {@link rx.util.functions.Func0} object, which will determine when this new window is emitted. When the source {@link rx.Observable} completes or produces an error, the current window * is emitted, and the event is propagated @@ -55,16 +53,16 @@ public Window call() { * The {@link rx.Observable} which produces values. * @param windowClosingSelector * A {@link rx.util.functions.Func0} object which produces {@link rx.Observable}s. These {@link rx.Observable}s determine when a window is emitted and replaced by simply - * producing an {@link rx.util.Closing} object. + * producing an object. * @return * the {@link rx.util.functions.Func1} object representing the specified window operation. */ - public static OnSubscribeFunc> window(final Observable source, final Func0> windowClosingSelector) { + public static OnSubscribeFunc> window(final Observable source, final Func0> windowClosingSelector) { return new OnSubscribeFunc>() { @Override public Subscription onSubscribe(final Observer> observer) { NonOverlappingChunks> windows = new NonOverlappingChunks>(observer, OperationWindow. windowMaker()); - ChunkCreator creator = new ObservableBasedSingleChunkCreator>(windows, windowClosingSelector); + ChunkCreator creator = new ObservableBasedSingleChunkCreator, TClosing>(windows, windowClosingSelector); return source.subscribe(new ChunkObserver>(windows, observer, creator)); } @@ -78,7 +76,7 @@ public Subscription onSubscribe(final Observer> observer) * *

Windows can be created by pushing a {@link rx.util.Opening} value to the "windowOpenings" {@link rx.Observable}. * This creates a new window which will then start recording values which are produced by the "source" {@link rx.Observable}. Additionally the "windowClosingSelector" will be used to construct an - * {@link rx.Observable} which can produce {@link rx.util.Closing} values. When it does so it will close this (and only this) newly created + * {@link rx.Observable} which can produce values. When it does so it will close this (and only this) newly created * window. When the source {@link rx.Observable} completes or produces an error, all windows are emitted, and the * event is propagated to all subscribed {@link rx.Observer}s.

* @@ -92,16 +90,16 @@ public Subscription onSubscribe(final Observer> observer) * create a new window which instantly starts recording the "source" {@link rx.Observable}. * @param windowClosingSelector * A {@link rx.util.functions.Func0} object which produces {@link rx.Observable}s. These {@link rx.Observable}s determine when a window is emitted and replaced by simply - * producing an {@link rx.util.Closing} object. + * producing an object. * @return * the {@link rx.util.functions.Func1} object representing the specified window operation. */ - public static OnSubscribeFunc> window(final Observable source, final Observable windowOpenings, final Func1> windowClosingSelector) { + public static OnSubscribeFunc> window(final Observable source, final Observable windowOpenings, final Func1> windowClosingSelector) { return new OnSubscribeFunc>() { @Override public Subscription onSubscribe(final Observer> observer) { OverlappingChunks> windows = new OverlappingChunks>(observer, OperationWindow. windowMaker()); - ChunkCreator creator = new ObservableBasedMultiChunkCreator>(windows, windowOpenings, windowClosingSelector); + ChunkCreator creator = new ObservableBasedMultiChunkCreator, TOpening, TClosing>(windows, windowOpenings, windowClosingSelector); return source.subscribe(new ChunkObserver>(windows, observer, creator)); } }; diff --git a/rxjava-core/src/main/java/rx/util/Closing.java b/rxjava-core/src/main/java/rx/util/Closing.java deleted file mode 100644 index 987d7a9dbe..0000000000 --- a/rxjava-core/src/main/java/rx/util/Closing.java +++ /dev/null @@ -1,20 +0,0 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package rx.util; - -public interface Closing { - // Tagging interface for objects which can close buffers. -} \ No newline at end of file diff --git a/rxjava-core/src/main/java/rx/util/Closings.java b/rxjava-core/src/main/java/rx/util/Closings.java deleted file mode 100644 index 0de43b97e9..0000000000 --- a/rxjava-core/src/main/java/rx/util/Closings.java +++ /dev/null @@ -1,28 +0,0 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package rx.util; - -public class Closings { - - public static Closing create() { - return new Closing() { - }; - } - - private Closings() { - // Prevent instantation. - } -} \ No newline at end of file diff --git a/rxjava-core/src/main/java/rx/util/Opening.java b/rxjava-core/src/main/java/rx/util/Opening.java deleted file mode 100644 index 03a0bbcfb1..0000000000 --- a/rxjava-core/src/main/java/rx/util/Opening.java +++ /dev/null @@ -1,20 +0,0 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package rx.util; - -public interface Opening { - // Tagging interface for objects which can open buffers. -} \ No newline at end of file diff --git a/rxjava-core/src/main/java/rx/util/Openings.java b/rxjava-core/src/main/java/rx/util/Openings.java deleted file mode 100644 index 30e11f72ca..0000000000 --- a/rxjava-core/src/main/java/rx/util/Openings.java +++ /dev/null @@ -1,28 +0,0 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package rx.util; - -public class Openings { - - public static Opening create() { - return new Opening() { - }; - } - - private Openings() { - // Prevent instantation. - } -} \ No newline at end of file diff --git a/rxjava-core/src/test/java/rx/operators/OperationBufferTest.java b/rxjava-core/src/test/java/rx/operators/OperationBufferTest.java index 9318af8fd6..475326a622 100644 --- a/rxjava-core/src/test/java/rx/operators/OperationBufferTest.java +++ b/rxjava-core/src/test/java/rx/operators/OperationBufferTest.java @@ -33,10 +33,6 @@ import rx.Subscription; import rx.concurrency.TestScheduler; import rx.subscriptions.Subscriptions; -import rx.util.Closing; -import rx.util.Closings; -import rx.util.Opening; -import rx.util.Openings; import rx.util.functions.Action0; import rx.util.functions.Action1; import rx.util.functions.Func0; @@ -226,23 +222,23 @@ public Subscription onSubscribe(Observer observer) { } }); - Observable openings = Observable.create(new Observable.OnSubscribeFunc() { + Observable openings = Observable.create(new Observable.OnSubscribeFunc() { @Override - public Subscription onSubscribe(Observer observer) { - push(observer, Openings.create(), 50); - push(observer, Openings.create(), 200); + public Subscription onSubscribe(Observer observer) { + push(observer, new Object(), 50); + push(observer, new Object(), 200); complete(observer, 250); return Subscriptions.empty(); } }); - Func1> closer = new Func1>() { + Func1> closer = new Func1>() { @Override - public Observable call(Opening opening) { - return Observable.create(new Observable.OnSubscribeFunc() { + public Observable call(Object opening) { + return Observable.create(new Observable.OnSubscribeFunc() { @Override - public Subscription onSubscribe(Observer observer) { - push(observer, Closings.create(), 100); + public Subscription onSubscribe(Observer observer) { + push(observer, new Object(), 100); complete(observer, 101); return Subscriptions.empty(); } @@ -277,13 +273,13 @@ public Subscription onSubscribe(Observer observer) { } }); - Func0> closer = new Func0>() { + Func0> closer = new Func0>() { @Override - public Observable call() { - return Observable.create(new Observable.OnSubscribeFunc() { + public Observable call() { + return Observable.create(new Observable.OnSubscribeFunc() { @Override - public Subscription onSubscribe(Observer observer) { - push(observer, Closings.create(), 100); + public Subscription onSubscribe(Observer observer) { + push(observer, new Object(), 100); complete(observer, 101); return Subscriptions.empty(); } diff --git a/rxjava-core/src/test/java/rx/operators/OperationWindowTest.java b/rxjava-core/src/test/java/rx/operators/OperationWindowTest.java index b26cf42cae..6eff582878 100644 --- a/rxjava-core/src/test/java/rx/operators/OperationWindowTest.java +++ b/rxjava-core/src/test/java/rx/operators/OperationWindowTest.java @@ -30,10 +30,6 @@ import rx.Subscription; import rx.concurrency.TestScheduler; import rx.subscriptions.Subscriptions; -import rx.util.Closing; -import rx.util.Closings; -import rx.util.Opening; -import rx.util.Openings; import rx.util.functions.Action0; import rx.util.functions.Action1; import rx.util.functions.Func0; @@ -202,23 +198,23 @@ public Subscription onSubscribe(Observer observer) { } }); - Observable openings = Observable.create(new Observable.OnSubscribeFunc() { + Observable openings = Observable.create(new Observable.OnSubscribeFunc() { @Override - public Subscription onSubscribe(Observer observer) { - push(observer, Openings.create(), 50); - push(observer, Openings.create(), 200); + public Subscription onSubscribe(Observer observer) { + push(observer, new Object(), 50); + push(observer, new Object(), 200); complete(observer, 250); return Subscriptions.empty(); } }); - Func1> closer = new Func1>() { + Func1> closer = new Func1>() { @Override - public Observable call(Opening opening) { - return Observable.create(new Observable.OnSubscribeFunc() { + public Observable call(Object opening) { + return Observable.create(new Observable.OnSubscribeFunc() { @Override - public Subscription onSubscribe(Observer observer) { - push(observer, Closings.create(), 100); + public Subscription onSubscribe(Observer observer) { + push(observer, new Object(), 100); complete(observer, 101); return Subscriptions.empty(); } @@ -253,13 +249,13 @@ public Subscription onSubscribe(Observer observer) { } }); - Func0> closer = new Func0>() { + Func0> closer = new Func0>() { @Override - public Observable call() { - return Observable.create(new Observable.OnSubscribeFunc() { + public Observable call() { + return Observable.create(new Observable.OnSubscribeFunc() { @Override - public Subscription onSubscribe(Observer observer) { - push(observer, Closings.create(), 100); + public Subscription onSubscribe(Observer observer) { + push(observer, new Object(), 100); complete(observer, 101); return Subscriptions.empty(); } From d85c15f98d544a8d08385881b2ef6b3a2186d49b Mon Sep 17 00:00:00 2001 From: akarnokd Date: Fri, 6 Dec 2013 12:35:21 +0100 Subject: [PATCH 05/15] Fixed scala compilation error. --- .../rxjava-scala/src/main/scala/rx/lang/scala/Observable.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/language-adaptors/rxjava-scala/src/main/scala/rx/lang/scala/Observable.scala b/language-adaptors/rxjava-scala/src/main/scala/rx/lang/scala/Observable.scala index b190f23aff..b0a6392cb4 100644 --- a/language-adaptors/rxjava-scala/src/main/scala/rx/lang/scala/Observable.scala +++ b/language-adaptors/rxjava-scala/src/main/scala/rx/lang/scala/Observable.scala @@ -528,7 +528,7 @@ trait Observable[+T] val o1: rx.Observable[_ <: rx.Observable[_]] = asJavaObservable.window[Closing](func) val o2 = Observable[rx.Observable[_]](o1).map((x: rx.Observable[_]) => { val x2 = x.asInstanceOf[rx.Observable[_ <: T]] - Observable[T](x2) + toScalaObservable[T](x2) }) o2 } From 0b15d20c2c5ec171aba36a8346cdb57fdfdaa843 Mon Sep 17 00:00:00 2001 From: samuelgruetter Date: Fri, 6 Dec 2013 13:25:54 +0100 Subject: [PATCH 06/15] remove package rx.lang.scala.util since all its contents were removed --- .../scala/rx/lang/scala/util/package.scala | 25 ------------------- 1 file changed, 25 deletions(-) delete mode 100644 language-adaptors/rxjava-scala/src/main/scala/rx/lang/scala/util/package.scala diff --git a/language-adaptors/rxjava-scala/src/main/scala/rx/lang/scala/util/package.scala b/language-adaptors/rxjava-scala/src/main/scala/rx/lang/scala/util/package.scala deleted file mode 100644 index 45afef4a4d..0000000000 --- a/language-adaptors/rxjava-scala/src/main/scala/rx/lang/scala/util/package.scala +++ /dev/null @@ -1,25 +0,0 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package rx.lang.scala - -/** - * Provides [[Opening]]s, [[Closing]]s, and [[Timestamped]]. - */ -package object util { - - // rx.util.Range not needed because there's a standard Scala Range - -} From b54faa563ae0c84ee3165cf20f2d5fe06e6f1ac5 Mon Sep 17 00:00:00 2001 From: samuelgruetter Date: Fri, 6 Dec 2013 13:36:43 +0100 Subject: [PATCH 07/15] remove import rx.lang.scala.util._ --- .../rxjava-scala/src/main/scala/rx/lang/scala/Observable.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/language-adaptors/rxjava-scala/src/main/scala/rx/lang/scala/Observable.scala b/language-adaptors/rxjava-scala/src/main/scala/rx/lang/scala/Observable.scala index b0a6392cb4..895e2ec672 100644 --- a/language-adaptors/rxjava-scala/src/main/scala/rx/lang/scala/Observable.scala +++ b/language-adaptors/rxjava-scala/src/main/scala/rx/lang/scala/Observable.scala @@ -75,7 +75,6 @@ trait Observable[+T] import scala.collection.Seq import scala.concurrent.duration.{Duration, TimeUnit} import rx.util.functions._ - import rx.lang.scala.util._ import rx.lang.scala.observables.BlockingObservable import ImplicitFunctionConversions._ import JavaConversions._ From e1a4d026101dee2c59b9b09dd514350af4e92079 Mon Sep 17 00:00:00 2001 From: samuelgruetter Date: Fri, 6 Dec 2013 14:48:02 +0100 Subject: [PATCH 08/15] clarify documentation on cache() --- .../src/main/scala/rx/lang/scala/Observable.scala | 6 +++++- rxjava-core/src/main/java/rx/Observable.java | 4 ++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/language-adaptors/rxjava-scala/src/main/scala/rx/lang/scala/Observable.scala b/language-adaptors/rxjava-scala/src/main/scala/rx/lang/scala/Observable.scala index 9999ed9751..98ddad076e 100644 --- a/language-adaptors/rxjava-scala/src/main/scala/rx/lang/scala/Observable.scala +++ b/language-adaptors/rxjava-scala/src/main/scala/rx/lang/scala/Observable.scala @@ -1015,7 +1015,11 @@ trait Observable[+T] * This is useful when you want an Observable to cache responses and you can't control the * subscribe/unsubscribe behavior of all the [[rx.lang.scala.Observer]]s. * - * NOTE: You sacrifice the ability to unsubscribe from the origin when you use the + * When you call `cache`, it does not yet subscribe to the + * source Observable. This only happens when `subscribe` is called + * the first time on the Observable returned by `cache()`. + * + * Note: You sacrifice the ability to unsubscribe from the origin when you use the * `cache()` operator so be careful not to use this operator on Observables that * emit an infinite or very large number of items that will use up memory. * diff --git a/rxjava-core/src/main/java/rx/Observable.java b/rxjava-core/src/main/java/rx/Observable.java index 9482a5f09f..59e55eb8d7 100644 --- a/rxjava-core/src/main/java/rx/Observable.java +++ b/rxjava-core/src/main/java/rx/Observable.java @@ -4218,6 +4218,10 @@ public Observable retry() { * can't control the subscribe/unsubscribe behavior of all the * {@link Observer}s. *

+ * When you call {@code cache()}, it does not yet subscribe to the + * source Observable. This only happens when {@code subscribe} is called + * the first time on the Observable returned by {@code cache()}. + *

* Note: You sacrifice the ability to unsubscribe from the origin when you * use the cache() operator so be careful not to use this * operator on Observables that emit an infinite or very large number of From 35d8005a59314a4912845f3df4d50d2759303230 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Fri, 6 Dec 2013 19:46:34 +0800 Subject: [PATCH 09/15] Reimplement the 'SequenceEqual' operator using other operators --- rxjava-core/src/main/java/rx/Observable.java | 21 +-- .../rx/operators/OperationSequenceEqual.java | 73 +++++++++ .../src/test/java/rx/ObservableTests.java | 11 -- .../OperationSequenceEqualTests.java | 145 ++++++++++++++++++ 4 files changed, 229 insertions(+), 21 deletions(-) create mode 100644 rxjava-core/src/main/java/rx/operators/OperationSequenceEqual.java create mode 100644 rxjava-core/src/test/java/rx/operators/OperationSequenceEqualTests.java diff --git a/rxjava-core/src/main/java/rx/Observable.java b/rxjava-core/src/main/java/rx/Observable.java index 9482a5f09f..fb1669efa4 100644 --- a/rxjava-core/src/main/java/rx/Observable.java +++ b/rxjava-core/src/main/java/rx/Observable.java @@ -76,6 +76,7 @@ import rx.operators.OperationRetry; import rx.operators.OperationSample; import rx.operators.OperationScan; +import rx.operators.OperationSequenceEqual; import rx.operators.OperationSkip; import rx.operators.OperationSkipLast; import rx.operators.OperationSkipUntil; @@ -2298,16 +2299,16 @@ public static Observable from(Future future, long timeout, T } /** - * Returns an Observable that emits Boolean values that indicate whether the - * pairs of items emitted by two source Observables are equal. + * Returns an Observable that emits a Boolean value that indicate + * whether two sequences are equal by comparing the elements pairwise. *

* * * @param first the first Observable to compare * @param second the second Observable to compare * @param the type of items emitted by each Observable - * @return an Observable that emits Booleans that indicate whether the - * corresponding items emitted by the source Observables are equal + * @return an Observable that emits a Boolean value that indicate + * whether two sequences are equal by comparing the elements pairwise. * @see RxJava Wiki: sequenceEqual() */ public static Observable sequenceEqual(Observable first, Observable second) { @@ -2320,9 +2321,9 @@ public Boolean call(T first, T second) { } /** - * Returns an Observable that emits Boolean values that indicate whether the - * pairs of items emitted by two source Observables are equal based on the - * results of a specified equality function. + * Returns an Observable that emits a Boolean value that indicate + * whether two sequences are equal by comparing the elements pairwise + * based on the results of a specified equality function. *

* * @@ -2331,12 +2332,12 @@ public Boolean call(T first, T second) { * @param equality a function used to compare items emitted by both * Observables * @param the type of items emitted by each Observable - * @return an Observable that emits Booleans that indicate whether the - * corresponding items emitted by the source Observables are equal + * @return an Observable that emits a Boolean value that indicate + * whether two sequences are equal by comparing the elements pairwise. * @see RxJava Wiki: sequenceEqual() */ public static Observable sequenceEqual(Observable first, Observable second, Func2 equality) { - return zip(first, second, equality); + return OperationSequenceEqual.sequenceEqual(first, second, equality); } /** diff --git a/rxjava-core/src/main/java/rx/operators/OperationSequenceEqual.java b/rxjava-core/src/main/java/rx/operators/OperationSequenceEqual.java new file mode 100644 index 0000000000..423ddbc358 --- /dev/null +++ b/rxjava-core/src/main/java/rx/operators/OperationSequenceEqual.java @@ -0,0 +1,73 @@ +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package rx.operators; + +import static rx.Observable.concat; +import static rx.Observable.from; +import static rx.Observable.zip; +import rx.Notification; +import rx.Observable; +import rx.util.functions.Func1; +import rx.util.functions.Func2; +import rx.util.functions.Functions; + +/** + * Returns an Observable that emits a Boolean value that indicate whether two + * sequences are equal by comparing the elements pairwise. + */ +public class OperationSequenceEqual { + + public static Observable sequenceEqual( + Observable first, Observable second, + final Func2 equality) { + Observable> firstObservable = concat( + first.map(new Func1>() { + + @Override + public Notification call(T t1) { + return new Notification(t1); + } + + }), from(new Notification())); + + Observable> secondObservable = concat( + second.map(new Func1>() { + + @Override + public Notification call(T t1) { + return new Notification(t1); + } + + }), from(new Notification())); + + return zip(firstObservable, secondObservable, + new Func2, Notification, Boolean>() { + + @Override + public Boolean call(Notification t1, Notification t2) { + if (t1.isOnCompleted() && t2.isOnCompleted()) { + return true; + } + if (t1.isOnCompleted() || t2.isOnCompleted()) { + return false; + } + // Now t1 and t2 must be 'onNext'. + return equality.call(t1.getValue(), t2.getValue()); + } + + }).all(Functions. identity()); + } +} diff --git a/rxjava-core/src/test/java/rx/ObservableTests.java b/rxjava-core/src/test/java/rx/ObservableTests.java index 3d718210a1..e4023f44ab 100644 --- a/rxjava-core/src/test/java/rx/ObservableTests.java +++ b/rxjava-core/src/test/java/rx/ObservableTests.java @@ -275,17 +275,6 @@ public Integer call(Integer t1, Integer t2) { verify(w).onNext(60); } - @Test - public void testSequenceEqual() { - Observable first = Observable.from(1, 2, 3); - Observable second = Observable.from(1, 2, 4); - @SuppressWarnings("unchecked") - Observer result = mock(Observer.class); - Observable.sequenceEqual(first, second).subscribe(result); - verify(result, times(2)).onNext(true); - verify(result, times(1)).onNext(false); - } - @Test public void testOnSubscribeFails() { @SuppressWarnings("unchecked") diff --git a/rxjava-core/src/test/java/rx/operators/OperationSequenceEqualTests.java b/rxjava-core/src/test/java/rx/operators/OperationSequenceEqualTests.java new file mode 100644 index 0000000000..a3cc8f1521 --- /dev/null +++ b/rxjava-core/src/test/java/rx/operators/OperationSequenceEqualTests.java @@ -0,0 +1,145 @@ +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package rx.operators; + +import static org.mockito.Matchers.isA; +import static org.mockito.Mockito.inOrder; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; + +import org.junit.Test; +import org.mockito.InOrder; + +import rx.Observable; +import rx.Observer; + +public class OperationSequenceEqualTests { + + @Test + public void test1() { + Observable observable = Observable.sequenceEqual( + Observable.from("one", "two", "three"), + Observable.from("one", "two", "three")); + verifyResult(observable, true); + } + + @Test + public void test2() { + Observable observable = Observable.sequenceEqual( + Observable.from("one", "two", "three"), + Observable.from("one", "two", "three", "four")); + verifyResult(observable, false); + } + + @Test + public void test3() { + Observable observable = Observable.sequenceEqual( + Observable.from("one", "two", "three", "four"), + Observable.from("one", "two", "three")); + verifyResult(observable, false); + } + + @Test + public void testWithError1() { + Observable observable = Observable.sequenceEqual( + Observable.concat(Observable.from("one"), + Observable. error(new TestException())), + Observable.from("one", "two", "three")); + verifyError(observable); + } + + @Test + public void testWithError2() { + Observable observable = Observable.sequenceEqual( + Observable.from("one", "two", "three"), + Observable.concat(Observable.from("one"), + Observable. error(new TestException()))); + verifyError(observable); + } + + @Test + public void testWithError3() { + Observable observable = Observable.sequenceEqual( + Observable.concat(Observable.from("one"), + Observable. error(new TestException())), + Observable.concat(Observable.from("one"), + Observable. error(new TestException()))); + verifyError(observable); + } + + @Test + public void testWithEmpty1() { + Observable observable = Observable.sequenceEqual( + Observable. empty(), + Observable.from("one", "two", "three")); + verifyResult(observable, false); + } + + @Test + public void testWithEmpty2() { + Observable observable = Observable.sequenceEqual( + Observable.from("one", "two", "three"), + Observable. empty()); + verifyResult(observable, false); + } + + @Test + public void testWithEmpty3() { + Observable observable = Observable.sequenceEqual( + Observable. empty(), Observable. empty()); + verifyResult(observable, true); + } + + @Test + public void testWithEqualityError() { + Observable observable = Observable.sequenceEqual( + Observable.from((String) null), Observable.from("one")); + + @SuppressWarnings("unchecked") + Observer observer = mock(Observer.class); + observable.subscribe(observer); + + InOrder inOrder = inOrder(observer); + inOrder.verify(observer, times(1)).onError( + isA(NullPointerException.class)); + inOrder.verifyNoMoreInteractions(); + } + + private void verifyResult(Observable observable, boolean result) { + @SuppressWarnings("unchecked") + Observer observer = mock(Observer.class); + observable.subscribe(observer); + + InOrder inOrder = inOrder(observer); + inOrder.verify(observer, times(1)).onNext(result); + inOrder.verify(observer).onCompleted(); + inOrder.verifyNoMoreInteractions(); + } + + private void verifyError(Observable observable) { + @SuppressWarnings("unchecked") + Observer observer = mock(Observer.class); + observable.subscribe(observer); + + InOrder inOrder = inOrder(observer); + inOrder.verify(observer, times(1)).onError(isA(TestException.class)); + inOrder.verifyNoMoreInteractions(); + } + + private class TestException extends RuntimeException { + private static final long serialVersionUID = 1L; + } +} From b753147c631841a4492fa5432e1f2ff86cc8a6ab Mon Sep 17 00:00:00 2001 From: zsxwing Date: Sat, 7 Dec 2013 23:01:17 +0800 Subject: [PATCH 10/15] Fix the 'null' issue in the default equality --- rxjava-core/src/main/java/rx/Observable.java | 3 ++ .../OperationSequenceEqualTests.java | 29 ++++++++++++++----- 2 files changed, 24 insertions(+), 8 deletions(-) diff --git a/rxjava-core/src/main/java/rx/Observable.java b/rxjava-core/src/main/java/rx/Observable.java index fb1669efa4..ff304d42e4 100644 --- a/rxjava-core/src/main/java/rx/Observable.java +++ b/rxjava-core/src/main/java/rx/Observable.java @@ -2315,6 +2315,9 @@ public static Observable sequenceEqual(Observable firs return sequenceEqual(first, second, new Func2() { @Override public Boolean call(T first, T second) { + if(first == null) { + return second == null; + } return first.equals(second); } }); diff --git a/rxjava-core/src/test/java/rx/operators/OperationSequenceEqualTests.java b/rxjava-core/src/test/java/rx/operators/OperationSequenceEqualTests.java index a3cc8f1521..71aa64130c 100644 --- a/rxjava-core/src/test/java/rx/operators/OperationSequenceEqualTests.java +++ b/rxjava-core/src/test/java/rx/operators/OperationSequenceEqualTests.java @@ -25,6 +25,7 @@ import rx.Observable; import rx.Observer; +import rx.util.functions.Func2; public class OperationSequenceEqualTests { @@ -104,18 +105,30 @@ public void testWithEmpty3() { } @Test - public void testWithEqualityError() { + public void testWithNull1() { Observable observable = Observable.sequenceEqual( Observable.from((String) null), Observable.from("one")); + verifyResult(observable, false); + } - @SuppressWarnings("unchecked") - Observer observer = mock(Observer.class); - observable.subscribe(observer); + @Test + public void testWithNull2() { + Observable observable = Observable.sequenceEqual( + Observable.from((String) null), Observable.from((String) null)); + verifyResult(observable, true); + } - InOrder inOrder = inOrder(observer); - inOrder.verify(observer, times(1)).onError( - isA(NullPointerException.class)); - inOrder.verifyNoMoreInteractions(); + @Test + public void testWithEqualityError() { + Observable observable = Observable.sequenceEqual( + Observable.from("one"), Observable.from("one"), + new Func2() { + @Override + public Boolean call(String t1, String t2) { + throw new TestException(); + } + }); + verifyError(observable); } private void verifyResult(Observable observable, boolean result) { From 01aab571de61bc07909384960dbc263c11ea4e71 Mon Sep 17 00:00:00 2001 From: headinthebox Date: Sat, 7 Dec 2013 17:36:18 -0800 Subject: [PATCH 11/15] Update ObservableHttp.java Removed pesty dangling

--- .../src/main/java/rx/apache/http/ObservableHttp.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rxjava-contrib/rxjava-apache-http/src/main/java/rx/apache/http/ObservableHttp.java b/rxjava-contrib/rxjava-apache-http/src/main/java/rx/apache/http/ObservableHttp.java index 7f34365eb4..1ed64ea838 100644 --- a/rxjava-contrib/rxjava-apache-http/src/main/java/rx/apache/http/ObservableHttp.java +++ b/rxjava-contrib/rxjava-apache-http/src/main/java/rx/apache/http/ObservableHttp.java @@ -112,7 +112,7 @@ public static ObservableHttp createGet(String uri, final *

* A client can be retrieved like this: *

- *

 {@code      CloseableHttpAsyncClient httpclient = HttpAsyncClients.createDefault(); } 

+ *
 {@code      CloseableHttpAsyncClient httpclient = HttpAsyncClients.createDefault(); } 
*

* A client with custom configurations can be created like this: *

From 21f7d526ad38506c0635576f54cc710c2e78a7f7 Mon Sep 17 00:00:00 2001 From: akarnokd Date: Sun, 8 Dec 2013 15:42:58 +0100 Subject: [PATCH 12/15] Fix Concat to allow multiple observers --- .../java/rx/operators/OperationConcat.java | 30 ++++++++++--- .../rx/operators/OperationConcatTest.java | 44 ++++++++++++++++++- 2 files changed, 66 insertions(+), 8 deletions(-) diff --git a/rxjava-core/src/main/java/rx/operators/OperationConcat.java b/rxjava-core/src/main/java/rx/operators/OperationConcat.java index 9838a8073b..e2c9be1ebf 100644 --- a/rxjava-core/src/main/java/rx/operators/OperationConcat.java +++ b/rxjava-core/src/main/java/rx/operators/OperationConcat.java @@ -51,7 +51,12 @@ public static OnSubscribeFunc concat(final Iterable OnSubscribeFunc concat(final Observable> sequences) { - return new Concat(sequences); + return new OnSubscribeFunc() { + @Override + public Subscription onSubscribe(Observer t1) { + return new Concat(sequences).onSubscribe(t1); + } + }; } private static class Concat implements OnSubscribeFunc { @@ -121,8 +126,12 @@ public void onNext(Observable nextSequence) { @Override public void onError(Throwable e) { if (completedOrErred.compareAndSet(false, true)) { - if (innerSubscription != null) { - innerSubscription.unsubscribe(); + Subscription q; + synchronized (nextSequences) { + q = innerSubscription; + } + if (q != null) { + q.unsubscribe(); } observer.onError(e); } @@ -131,7 +140,11 @@ public void onError(Throwable e) { @Override public void onCompleted() { allSequencesReceived.set(true); - if (innerSubscription == null) { + Subscription q; + synchronized (nextSequences) { + q = innerSubscription; + } + if (q == null) { // We are not subscribed to any sequence, and none are coming anymore if (completedOrErred.compareAndSet(false, true)) { observer.onCompleted(); @@ -143,11 +156,14 @@ public void onCompleted() { return new Subscription() { @Override public void unsubscribe() { + Subscription q; synchronized (nextSequences) { - if (innerSubscription != null) - innerSubscription.unsubscribe(); - outerSubscription.unsubscribe(); + q = innerSubscription; + } + if (q != null) { + q.unsubscribe(); } + outerSubscription.unsubscribe(); } }; } diff --git a/rxjava-core/src/test/java/rx/operators/OperationConcatTest.java b/rxjava-core/src/test/java/rx/operators/OperationConcatTest.java index f42d8b81ad..0f3f8f8866 100644 --- a/rxjava-core/src/test/java/rx/operators/OperationConcatTest.java +++ b/rxjava-core/src/test/java/rx/operators/OperationConcatTest.java @@ -16,7 +16,6 @@ package rx.operators; import static org.junit.Assert.*; -import static org.mockito.Matchers.*; import static org.mockito.Mockito.*; import static rx.operators.OperationConcat.*; @@ -33,6 +32,7 @@ import rx.Observable; import rx.Observer; import rx.Subscription; +import rx.concurrency.TestScheduler; import rx.subscriptions.BooleanSubscription; public class OperationConcatTest { @@ -556,4 +556,46 @@ public void run() { return s; } } + @Test + public void testMultipleObservers() { + Observer o1 = mock(Observer.class); + Observer o2 = mock(Observer.class); + + TestScheduler s = new TestScheduler(); + + Observable timer = Observable.interval(500, TimeUnit.MILLISECONDS, s).take(2); + Observable o = Observable.concat(timer, timer); + + o.subscribe(o1); + o.subscribe(o2); + + InOrder inOrder1 = inOrder(o1); + InOrder inOrder2 = inOrder(o2); + + s.advanceTimeBy(500, TimeUnit.MILLISECONDS); + + inOrder1.verify(o1, times(1)).onNext(0L); + inOrder2.verify(o2, times(1)).onNext(0L); + + s.advanceTimeBy(500, TimeUnit.MILLISECONDS); + + inOrder1.verify(o1, times(1)).onNext(1L); + inOrder2.verify(o2, times(1)).onNext(1L); + + s.advanceTimeBy(500, TimeUnit.MILLISECONDS); + + inOrder1.verify(o1, times(1)).onNext(0L); + inOrder2.verify(o2, times(1)).onNext(0L); + + s.advanceTimeBy(500, TimeUnit.MILLISECONDS); + + inOrder1.verify(o1, times(1)).onNext(1L); + inOrder2.verify(o2, times(1)).onNext(1L); + + inOrder1.verify(o1, times(1)).onCompleted(); + inOrder2.verify(o2, times(1)).onCompleted(); + + verify(o1, never()).onError(any(Throwable.class)); + verify(o2, never()).onError(any(Throwable.class)); + } } From 26a034939464122480d313d287e9ec2979c44613 Mon Sep 17 00:00:00 2001 From: akarnokd Date: Sun, 8 Dec 2013 16:25:26 +0100 Subject: [PATCH 13/15] Operation LongCount --- rxjava-core/src/main/java/rx/Observable.java | 22 ++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/rxjava-core/src/main/java/rx/Observable.java b/rxjava-core/src/main/java/rx/Observable.java index 9482a5f09f..5bc1c7eba2 100644 --- a/rxjava-core/src/main/java/rx/Observable.java +++ b/rxjava-core/src/main/java/rx/Observable.java @@ -3871,6 +3871,7 @@ public Observable reduce(Func2 accumulator) { * source Observable as its single item * @see RxJava Wiki: count() * @see MSDN: Observable.Count + * @see #longCount() */ public Observable count() { return reduce(0, new Func2() { @@ -5172,6 +5173,27 @@ public Observable last() { return create(OperationLast.last(this)); } +/** + * Returns an Observable that counts the total number of items in the + * source Observable as a 64 bit long. + *

+ * + * + * @return an Observable that emits the number of counted elements of the + * source Observable as its single, 64 bit long item + * @see RxJava Wiki: count() + * @see MSDN: Observable.LongCount + * @see #count() + */ + public Observable longCount() { + return reduce(0L, new Func2() { + @Override + public Long call(Long t1, T t2) { + return t1 + 1; + } + }); + } + /** * Converts an Observable into a {@link BlockingObservable} (an Observable * with blocking operators). From 4782588aa15a954ea053977d6b740127d31ee3f1 Mon Sep 17 00:00:00 2001 From: Ben Christensen Date: Sun, 8 Dec 2013 15:54:46 -0800 Subject: [PATCH 14/15] GitAttributes for Line Endings https://help.github.com/articles/dealing-with-line-endings See https://github.com/Netflix/RxJava/issues/579 --- .gitattributes | 15 +++++++++++++++ 1 file changed, 15 insertions(+) create mode 100644 .gitattributes diff --git a/.gitattributes b/.gitattributes new file mode 100644 index 0000000000..76e9aa3c6d --- /dev/null +++ b/.gitattributes @@ -0,0 +1,15 @@ +# Set default behaviour, in case users don't have core.autocrlf set. +* text=auto + +# Explicitly declare text files we want to always be normalized and converted +# to native line endings on checkout. +*.java text +*.groovy text +*.scala text +*.clj text +*.txt text +*.md text + +# Denote all files that are truly binary and should not be modified. +*.png binary +*.jpg binary From ceeab3696fa60123b49a3cc71b4e8c7b84a17f58 Mon Sep 17 00:00:00 2001 From: Ben Christensen Date: Sun, 8 Dec 2013 15:57:18 -0800 Subject: [PATCH 15/15] Normalize Line Endings See https://help.github.com/articles/dealing-with-line-endings#re-normalizing-a-repository Fixes https://github.com/Netflix/RxJava/issues/579 --- gradlew.bat | 180 +- .../src/main/java/rx/joins/ActivePlan1.java | 98 +- .../src/main/java/rx/joins/ActivePlan2.java | 108 +- .../src/main/java/rx/joins/ActivePlan3.java | 128 +- .../src/main/java/rx/joins/JoinObserver.java | 52 +- .../src/main/java/rx/joins/Pattern.java | 50 +- .../src/main/java/rx/joins/Pattern1.java | 90 +- .../src/main/java/rx/joins/Pattern2.java | 108 +- .../src/main/java/rx/joins/Pattern3.java | 110 +- rxjava-core/src/main/java/rx/joins/Plan2.java | 158 +- rxjava-core/src/main/java/rx/joins/Plan3.java | 166 +- .../rx/operators/OperationJoinPatterns.java | 262 +-- .../java/rx/operators/OperationToMap.java | 318 +-- .../rx/operators/OperationToMultimap.java | 412 ++-- .../rx/subscriptions/SerialSubscription.java | 124 +- .../SingleAssignmentSubscription.java | 162 +- .../main/java/rx/util/functions/Action4.java | 48 +- .../main/java/rx/util/functions/Action5.java | 48 +- .../main/java/rx/util/functions/Action6.java | 48 +- .../main/java/rx/util/functions/Action7.java | 46 +- .../main/java/rx/util/functions/Action8.java | 46 +- .../main/java/rx/util/functions/Action9.java | 46 +- .../main/java/rx/util/functions/ActionN.java | 46 +- .../main/java/rx/util/functions/Actions.java | 718 +++---- .../main/java/rx/util/functions/Async.java | 1914 ++++++++--------- .../java/rx/operators/OperationJoinsTest.java | 764 +++---- .../java/rx/operators/OperationToMapTest.java | 430 ++-- .../rx/operators/OperationToMultimapTest.java | 500 ++--- .../SerialSubscriptionTests.java | 422 ++-- .../java/rx/util/functions/AsyncTest.java | 1306 +++++------ 30 files changed, 4454 insertions(+), 4454 deletions(-) diff --git a/gradlew.bat b/gradlew.bat index aec99730b4..8a0b282aa6 100644 --- a/gradlew.bat +++ b/gradlew.bat @@ -1,90 +1,90 @@ -@if "%DEBUG%" == "" @echo off -@rem ########################################################################## -@rem -@rem Gradle startup script for Windows -@rem -@rem ########################################################################## - -@rem Set local scope for the variables with windows NT shell -if "%OS%"=="Windows_NT" setlocal - -@rem Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. -set DEFAULT_JVM_OPTS= - -set DIRNAME=%~dp0 -if "%DIRNAME%" == "" set DIRNAME=. -set APP_BASE_NAME=%~n0 -set APP_HOME=%DIRNAME% - -@rem Find java.exe -if defined JAVA_HOME goto findJavaFromJavaHome - -set JAVA_EXE=java.exe -%JAVA_EXE% -version >NUL 2>&1 -if "%ERRORLEVEL%" == "0" goto init - -echo. -echo ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. -echo. -echo Please set the JAVA_HOME variable in your environment to match the -echo location of your Java installation. - -goto fail - -:findJavaFromJavaHome -set JAVA_HOME=%JAVA_HOME:"=% -set JAVA_EXE=%JAVA_HOME%/bin/java.exe - -if exist "%JAVA_EXE%" goto init - -echo. -echo ERROR: JAVA_HOME is set to an invalid directory: %JAVA_HOME% -echo. -echo Please set the JAVA_HOME variable in your environment to match the -echo location of your Java installation. - -goto fail - -:init -@rem Get command-line arguments, handling Windowz variants - -if not "%OS%" == "Windows_NT" goto win9xME_args -if "%@eval[2+2]" == "4" goto 4NT_args - -:win9xME_args -@rem Slurp the command line arguments. -set CMD_LINE_ARGS= -set _SKIP=2 - -:win9xME_args_slurp -if "x%~1" == "x" goto execute - -set CMD_LINE_ARGS=%* -goto execute - -:4NT_args -@rem Get arguments from the 4NT Shell from JP Software -set CMD_LINE_ARGS=%$ - -:execute -@rem Setup the command line - -set CLASSPATH=%APP_HOME%\gradle\wrapper\gradle-wrapper.jar - -@rem Execute Gradle -"%JAVA_EXE%" %DEFAULT_JVM_OPTS% %JAVA_OPTS% %GRADLE_OPTS% "-Dorg.gradle.appname=%APP_BASE_NAME%" -classpath "%CLASSPATH%" org.gradle.wrapper.GradleWrapperMain %CMD_LINE_ARGS% - -:end -@rem End local scope for the variables with windows NT shell -if "%ERRORLEVEL%"=="0" goto mainEnd - -:fail -rem Set variable GRADLE_EXIT_CONSOLE if you need the _script_ return code instead of -rem the _cmd.exe /c_ return code! -if not "" == "%GRADLE_EXIT_CONSOLE%" exit 1 -exit /b 1 - -:mainEnd -if "%OS%"=="Windows_NT" endlocal - -:omega +@if "%DEBUG%" == "" @echo off +@rem ########################################################################## +@rem +@rem Gradle startup script for Windows +@rem +@rem ########################################################################## + +@rem Set local scope for the variables with windows NT shell +if "%OS%"=="Windows_NT" setlocal + +@rem Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. +set DEFAULT_JVM_OPTS= + +set DIRNAME=%~dp0 +if "%DIRNAME%" == "" set DIRNAME=. +set APP_BASE_NAME=%~n0 +set APP_HOME=%DIRNAME% + +@rem Find java.exe +if defined JAVA_HOME goto findJavaFromJavaHome + +set JAVA_EXE=java.exe +%JAVA_EXE% -version >NUL 2>&1 +if "%ERRORLEVEL%" == "0" goto init + +echo. +echo ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. +echo. +echo Please set the JAVA_HOME variable in your environment to match the +echo location of your Java installation. + +goto fail + +:findJavaFromJavaHome +set JAVA_HOME=%JAVA_HOME:"=% +set JAVA_EXE=%JAVA_HOME%/bin/java.exe + +if exist "%JAVA_EXE%" goto init + +echo. +echo ERROR: JAVA_HOME is set to an invalid directory: %JAVA_HOME% +echo. +echo Please set the JAVA_HOME variable in your environment to match the +echo location of your Java installation. + +goto fail + +:init +@rem Get command-line arguments, handling Windowz variants + +if not "%OS%" == "Windows_NT" goto win9xME_args +if "%@eval[2+2]" == "4" goto 4NT_args + +:win9xME_args +@rem Slurp the command line arguments. +set CMD_LINE_ARGS= +set _SKIP=2 + +:win9xME_args_slurp +if "x%~1" == "x" goto execute + +set CMD_LINE_ARGS=%* +goto execute + +:4NT_args +@rem Get arguments from the 4NT Shell from JP Software +set CMD_LINE_ARGS=%$ + +:execute +@rem Setup the command line + +set CLASSPATH=%APP_HOME%\gradle\wrapper\gradle-wrapper.jar + +@rem Execute Gradle +"%JAVA_EXE%" %DEFAULT_JVM_OPTS% %JAVA_OPTS% %GRADLE_OPTS% "-Dorg.gradle.appname=%APP_BASE_NAME%" -classpath "%CLASSPATH%" org.gradle.wrapper.GradleWrapperMain %CMD_LINE_ARGS% + +:end +@rem End local scope for the variables with windows NT shell +if "%ERRORLEVEL%"=="0" goto mainEnd + +:fail +rem Set variable GRADLE_EXIT_CONSOLE if you need the _script_ return code instead of +rem the _cmd.exe /c_ return code! +if not "" == "%GRADLE_EXIT_CONSOLE%" exit 1 +exit /b 1 + +:mainEnd +if "%OS%"=="Windows_NT" endlocal + +:omega diff --git a/rxjava-core/src/main/java/rx/joins/ActivePlan1.java b/rxjava-core/src/main/java/rx/joins/ActivePlan1.java index 65e50e9908..d5181b0982 100644 --- a/rxjava-core/src/main/java/rx/joins/ActivePlan1.java +++ b/rxjava-core/src/main/java/rx/joins/ActivePlan1.java @@ -1,49 +1,49 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package rx.joins; - -import rx.Notification; -import rx.util.functions.Action0; -import rx.util.functions.Action1; - -/** - * Represents an active plan. - */ -public class ActivePlan1 extends ActivePlan0 { - private final Action1 onNext; - private final Action0 onCompleted; - private final JoinObserver1 first; - public ActivePlan1(JoinObserver1 first, Action1 onNext, Action0 onCompleted) { - this.onNext = onNext; - this.onCompleted = onCompleted; - this.first = first; - addJoinObserver(first); - } - - @Override - public void match() { - if (!first.queue().isEmpty()) { - Notification n1 = first.queue().peek(); - if (n1.isOnCompleted()) { - onCompleted.call(); - } else { - dequeue(); - onNext.call(n1.getValue()); - } - } - } - -} +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ +package rx.joins; + +import rx.Notification; +import rx.util.functions.Action0; +import rx.util.functions.Action1; + +/** + * Represents an active plan. + */ +public class ActivePlan1 extends ActivePlan0 { + private final Action1 onNext; + private final Action0 onCompleted; + private final JoinObserver1 first; + public ActivePlan1(JoinObserver1 first, Action1 onNext, Action0 onCompleted) { + this.onNext = onNext; + this.onCompleted = onCompleted; + this.first = first; + addJoinObserver(first); + } + + @Override + public void match() { + if (!first.queue().isEmpty()) { + Notification n1 = first.queue().peek(); + if (n1.isOnCompleted()) { + onCompleted.call(); + } else { + dequeue(); + onNext.call(n1.getValue()); + } + } + } + +} diff --git a/rxjava-core/src/main/java/rx/joins/ActivePlan2.java b/rxjava-core/src/main/java/rx/joins/ActivePlan2.java index 3ae19714ce..a91d4fd8f3 100644 --- a/rxjava-core/src/main/java/rx/joins/ActivePlan2.java +++ b/rxjava-core/src/main/java/rx/joins/ActivePlan2.java @@ -1,54 +1,54 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package rx.joins; - -import rx.Notification; -import rx.util.functions.Action0; -import rx.util.functions.Action2; - -/** - * Represents an active plan. - */ -public class ActivePlan2 extends ActivePlan0 { - private final Action2 onNext; - private final Action0 onCompleted; - private final JoinObserver1 first; - private final JoinObserver1 second; - public ActivePlan2(JoinObserver1 first, JoinObserver1 second, Action2 onNext, Action0 onCompleted) { - this.onNext = onNext; - this.onCompleted = onCompleted; - this.first = first; - this.second = second; - addJoinObserver(first); - addJoinObserver(second); - } - - @Override - public void match() { - if (!first.queue().isEmpty() && !second.queue().isEmpty()) { - Notification n1 = first.queue().peek(); - Notification n2 = second.queue().peek(); - - if (n1.isOnCompleted() || n2.isOnCompleted()) { - onCompleted.call(); - } else { - dequeue(); - onNext.call(n1.getValue(), n2.getValue()); - } - } - } - -} +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ +package rx.joins; + +import rx.Notification; +import rx.util.functions.Action0; +import rx.util.functions.Action2; + +/** + * Represents an active plan. + */ +public class ActivePlan2 extends ActivePlan0 { + private final Action2 onNext; + private final Action0 onCompleted; + private final JoinObserver1 first; + private final JoinObserver1 second; + public ActivePlan2(JoinObserver1 first, JoinObserver1 second, Action2 onNext, Action0 onCompleted) { + this.onNext = onNext; + this.onCompleted = onCompleted; + this.first = first; + this.second = second; + addJoinObserver(first); + addJoinObserver(second); + } + + @Override + public void match() { + if (!first.queue().isEmpty() && !second.queue().isEmpty()) { + Notification n1 = first.queue().peek(); + Notification n2 = second.queue().peek(); + + if (n1.isOnCompleted() || n2.isOnCompleted()) { + onCompleted.call(); + } else { + dequeue(); + onNext.call(n1.getValue(), n2.getValue()); + } + } + } + +} diff --git a/rxjava-core/src/main/java/rx/joins/ActivePlan3.java b/rxjava-core/src/main/java/rx/joins/ActivePlan3.java index cff23dd752..36bfa0e09e 100644 --- a/rxjava-core/src/main/java/rx/joins/ActivePlan3.java +++ b/rxjava-core/src/main/java/rx/joins/ActivePlan3.java @@ -1,64 +1,64 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package rx.joins; - -import rx.Notification; -import rx.util.functions.Action0; -import rx.util.functions.Action3; - -/** - * Represents an active plan. - */ -public class ActivePlan3 extends ActivePlan0 { - private final Action3 onNext; - private final Action0 onCompleted; - private final JoinObserver1 first; - private final JoinObserver1 second; - private final JoinObserver1 third; - public ActivePlan3(JoinObserver1 first, - JoinObserver1 second, - JoinObserver1 third, - Action3 onNext, - Action0 onCompleted) { - this.onNext = onNext; - this.onCompleted = onCompleted; - this.first = first; - this.second = second; - this.third = third; - addJoinObserver(first); - addJoinObserver(second); - addJoinObserver(third); - } - - @Override - public void match() { - if (!first.queue().isEmpty() - && !second.queue().isEmpty() - && !third.queue().isEmpty()) { - Notification n1 = first.queue().peek(); - Notification n2 = second.queue().peek(); - Notification n3 = third.queue().peek(); - - if (n1.isOnCompleted() || n2.isOnCompleted() || n3.isOnCompleted()) { - onCompleted.call(); - } else { - dequeue(); - onNext.call(n1.getValue(), n2.getValue(), n3.getValue()); - } - } - } - -} +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ +package rx.joins; + +import rx.Notification; +import rx.util.functions.Action0; +import rx.util.functions.Action3; + +/** + * Represents an active plan. + */ +public class ActivePlan3 extends ActivePlan0 { + private final Action3 onNext; + private final Action0 onCompleted; + private final JoinObserver1 first; + private final JoinObserver1 second; + private final JoinObserver1 third; + public ActivePlan3(JoinObserver1 first, + JoinObserver1 second, + JoinObserver1 third, + Action3 onNext, + Action0 onCompleted) { + this.onNext = onNext; + this.onCompleted = onCompleted; + this.first = first; + this.second = second; + this.third = third; + addJoinObserver(first); + addJoinObserver(second); + addJoinObserver(third); + } + + @Override + public void match() { + if (!first.queue().isEmpty() + && !second.queue().isEmpty() + && !third.queue().isEmpty()) { + Notification n1 = first.queue().peek(); + Notification n2 = second.queue().peek(); + Notification n3 = third.queue().peek(); + + if (n1.isOnCompleted() || n2.isOnCompleted() || n3.isOnCompleted()) { + onCompleted.call(); + } else { + dequeue(); + onNext.call(n1.getValue(), n2.getValue(), n3.getValue()); + } + } + } + +} diff --git a/rxjava-core/src/main/java/rx/joins/JoinObserver.java b/rxjava-core/src/main/java/rx/joins/JoinObserver.java index d191ed8958..2557427895 100644 --- a/rxjava-core/src/main/java/rx/joins/JoinObserver.java +++ b/rxjava-core/src/main/java/rx/joins/JoinObserver.java @@ -1,26 +1,26 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package rx.joins; - -import rx.Subscription; - -/** - * Base interface to manage joined observations. - */ -public interface JoinObserver extends Subscription { - void subscribe(Object gate); - void dequeue(); -} +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ +package rx.joins; + +import rx.Subscription; + +/** + * Base interface to manage joined observations. + */ +public interface JoinObserver extends Subscription { + void subscribe(Object gate); + void dequeue(); +} diff --git a/rxjava-core/src/main/java/rx/joins/Pattern.java b/rxjava-core/src/main/java/rx/joins/Pattern.java index 1dc65ca3b8..cce00a3af7 100644 --- a/rxjava-core/src/main/java/rx/joins/Pattern.java +++ b/rxjava-core/src/main/java/rx/joins/Pattern.java @@ -1,25 +1,25 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package rx.joins; - -/** - * Base interface for join patterns. - * @see MSDN: Pattern - */ -public interface Pattern { - -} +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package rx.joins; + +/** + * Base interface for join patterns. + * @see MSDN: Pattern + */ +public interface Pattern { + +} diff --git a/rxjava-core/src/main/java/rx/joins/Pattern1.java b/rxjava-core/src/main/java/rx/joins/Pattern1.java index 324daf79fa..77600647e8 100644 --- a/rxjava-core/src/main/java/rx/joins/Pattern1.java +++ b/rxjava-core/src/main/java/rx/joins/Pattern1.java @@ -1,45 +1,45 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package rx.joins; - -import rx.Observable; -import rx.util.functions.Func1; - -/** - * Represents a join pattern over one observable sequence. - */ -public class Pattern1 implements Pattern { - private final Observable first; - public Pattern1(Observable first) { - this.first = first; - } - public Observable first() { - return first; - } - /** - * Matches when all observable sequences have an available - * element and projects the elements by invoking the selector function. - * @param selector the function that will be invoked for elements in the source sequences. - * @return - * @throws NullPointerException if selector is null - */ - public Plan0 then(Func1 selector) { - if (selector == null) { - throw new NullPointerException(); - } - return new Plan1(this, selector); - } -} +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ +package rx.joins; + +import rx.Observable; +import rx.util.functions.Func1; + +/** + * Represents a join pattern over one observable sequence. + */ +public class Pattern1 implements Pattern { + private final Observable first; + public Pattern1(Observable first) { + this.first = first; + } + public Observable first() { + return first; + } + /** + * Matches when all observable sequences have an available + * element and projects the elements by invoking the selector function. + * @param selector the function that will be invoked for elements in the source sequences. + * @return + * @throws NullPointerException if selector is null + */ + public Plan0 then(Func1 selector) { + if (selector == null) { + throw new NullPointerException(); + } + return new Plan1(this, selector); + } +} diff --git a/rxjava-core/src/main/java/rx/joins/Pattern2.java b/rxjava-core/src/main/java/rx/joins/Pattern2.java index 6f82a66d49..b38ba37253 100644 --- a/rxjava-core/src/main/java/rx/joins/Pattern2.java +++ b/rxjava-core/src/main/java/rx/joins/Pattern2.java @@ -1,54 +1,54 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package rx.joins; - -import rx.Observable; -import rx.util.functions.Func2; - -/** - * Represents a join pattern over observable sequences. - */ -public class Pattern2 implements Pattern { - private final Observable first; - private final Observable second; - public Pattern2(Observable first, Observable second) { - this.first = first; - this.second = second; - } - public Observable first() { - return first; - } - public Observable second() { - return second; - } - /** - * Creates a pattern that matches when all three observable sequences have an available element. - * @param other Observable sequence to match with the two previous sequences. - * @return Pattern object that matches when all observable sequences have an available element. - */ - public Pattern3 and(Observable other) { - if (other == null) { - throw new NullPointerException(); - } - return new Pattern3(first, second, other); - } - public Plan0 then(Func2 selector) { - if (selector == null) { - throw new NullPointerException(); - } - return new Plan2(this, selector); - } -} +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ +package rx.joins; + +import rx.Observable; +import rx.util.functions.Func2; + +/** + * Represents a join pattern over observable sequences. + */ +public class Pattern2 implements Pattern { + private final Observable first; + private final Observable second; + public Pattern2(Observable first, Observable second) { + this.first = first; + this.second = second; + } + public Observable first() { + return first; + } + public Observable second() { + return second; + } + /** + * Creates a pattern that matches when all three observable sequences have an available element. + * @param other Observable sequence to match with the two previous sequences. + * @return Pattern object that matches when all observable sequences have an available element. + */ + public Pattern3 and(Observable other) { + if (other == null) { + throw new NullPointerException(); + } + return new Pattern3(first, second, other); + } + public Plan0 then(Func2 selector) { + if (selector == null) { + throw new NullPointerException(); + } + return new Plan2(this, selector); + } +} diff --git a/rxjava-core/src/main/java/rx/joins/Pattern3.java b/rxjava-core/src/main/java/rx/joins/Pattern3.java index c43b9d0b9e..0871b3d3c2 100644 --- a/rxjava-core/src/main/java/rx/joins/Pattern3.java +++ b/rxjava-core/src/main/java/rx/joins/Pattern3.java @@ -1,55 +1,55 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package rx.joins; - -import rx.Observable; -import rx.util.functions.Func3; - -/** - * Represents a join pattern over observable sequences. - */ -public class Pattern3 implements Pattern { - private final Observable first; - private final Observable second; - private final Observable third; - public Pattern3(Observable first, Observable second, - Observable third) { - this.first = first; - this.second = second; - this.third = third; - } - public Observable first() { - return first; - } - public Observable second() { - return second; - } - public Observable third() { - return third; - } -// public Pattern4 and(Observable other) { -// if (other == null) { -// throw new NullPointerException(); -// } -// return new Pattern4(first, second, third, other); -// } - public Plan0 then(Func3 selector) { - if (selector == null) { - throw new NullPointerException(); - } - return new Plan3(this, selector); - } -} +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ +package rx.joins; + +import rx.Observable; +import rx.util.functions.Func3; + +/** + * Represents a join pattern over observable sequences. + */ +public class Pattern3 implements Pattern { + private final Observable first; + private final Observable second; + private final Observable third; + public Pattern3(Observable first, Observable second, + Observable third) { + this.first = first; + this.second = second; + this.third = third; + } + public Observable first() { + return first; + } + public Observable second() { + return second; + } + public Observable third() { + return third; + } +// public Pattern4 and(Observable other) { +// if (other == null) { +// throw new NullPointerException(); +// } +// return new Pattern4(first, second, third, other); +// } + public Plan0 then(Func3 selector) { + if (selector == null) { + throw new NullPointerException(); + } + return new Plan3(this, selector); + } +} diff --git a/rxjava-core/src/main/java/rx/joins/Plan2.java b/rxjava-core/src/main/java/rx/joins/Plan2.java index 71b931ea0d..456e3c2bb3 100644 --- a/rxjava-core/src/main/java/rx/joins/Plan2.java +++ b/rxjava-core/src/main/java/rx/joins/Plan2.java @@ -1,79 +1,79 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package rx.joins; - -import java.util.Map; -import java.util.concurrent.atomic.AtomicReference; -import rx.Observer; -import static rx.joins.Plan0.createObserver; -import rx.util.functions.Action0; -import rx.util.functions.Action1; -import rx.util.functions.Action2; -import rx.util.functions.Actions; -import rx.util.functions.Func2; - -/** - * Represents an execution plan for join patterns. - */ -public class Plan2 extends Plan0 { - protected Pattern2 expression; - protected Func2 selector; - public Plan2(Pattern2 expression, Func2 selector) { - this.expression = expression; - this.selector = selector; - } - - @Override - public ActivePlan0 activate(Map externalSubscriptions, - final Observer observer, final Action1 deactivate) { - Action1 onError = Actions.onErrorFrom(observer); - - final JoinObserver1 firstJoinObserver = createObserver(externalSubscriptions, expression.first(), onError); - final JoinObserver1 secondJoinObserver = createObserver(externalSubscriptions, expression.second(), onError); - - final AtomicReference> self = new AtomicReference>(); - - ActivePlan2 activePlan = new ActivePlan2(firstJoinObserver, secondJoinObserver, new Action2() { - @Override - public void call(T1 t1, T2 t2) { - R result; - try { - result = selector.call(t1, t2); - } catch (Throwable t) { - observer.onError(t); - return; - } - observer.onNext(result); - } - }, - new Action0() { - @Override - public void call() { - firstJoinObserver.removeActivePlan(self.get()); - secondJoinObserver.removeActivePlan(self.get()); - deactivate.call(self.get()); - } - }); - - self.set(activePlan); - - firstJoinObserver.addActivePlan(activePlan); - secondJoinObserver.addActivePlan(activePlan); - - return activePlan; - } - -} +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ +package rx.joins; + +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; +import rx.Observer; +import static rx.joins.Plan0.createObserver; +import rx.util.functions.Action0; +import rx.util.functions.Action1; +import rx.util.functions.Action2; +import rx.util.functions.Actions; +import rx.util.functions.Func2; + +/** + * Represents an execution plan for join patterns. + */ +public class Plan2 extends Plan0 { + protected Pattern2 expression; + protected Func2 selector; + public Plan2(Pattern2 expression, Func2 selector) { + this.expression = expression; + this.selector = selector; + } + + @Override + public ActivePlan0 activate(Map externalSubscriptions, + final Observer observer, final Action1 deactivate) { + Action1 onError = Actions.onErrorFrom(observer); + + final JoinObserver1 firstJoinObserver = createObserver(externalSubscriptions, expression.first(), onError); + final JoinObserver1 secondJoinObserver = createObserver(externalSubscriptions, expression.second(), onError); + + final AtomicReference> self = new AtomicReference>(); + + ActivePlan2 activePlan = new ActivePlan2(firstJoinObserver, secondJoinObserver, new Action2() { + @Override + public void call(T1 t1, T2 t2) { + R result; + try { + result = selector.call(t1, t2); + } catch (Throwable t) { + observer.onError(t); + return; + } + observer.onNext(result); + } + }, + new Action0() { + @Override + public void call() { + firstJoinObserver.removeActivePlan(self.get()); + secondJoinObserver.removeActivePlan(self.get()); + deactivate.call(self.get()); + } + }); + + self.set(activePlan); + + firstJoinObserver.addActivePlan(activePlan); + secondJoinObserver.addActivePlan(activePlan); + + return activePlan; + } + +} diff --git a/rxjava-core/src/main/java/rx/joins/Plan3.java b/rxjava-core/src/main/java/rx/joins/Plan3.java index 386ef82dfe..6f12235c51 100644 --- a/rxjava-core/src/main/java/rx/joins/Plan3.java +++ b/rxjava-core/src/main/java/rx/joins/Plan3.java @@ -1,83 +1,83 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package rx.joins; - -import java.util.Map; -import java.util.concurrent.atomic.AtomicReference; -import rx.Observer; -import static rx.joins.Plan0.createObserver; -import rx.util.functions.Action0; -import rx.util.functions.Action1; -import rx.util.functions.Action3; -import rx.util.functions.Actions; -import rx.util.functions.Func3; - -/** - * Represents an execution plan for join patterns. - */ -public class Plan3 extends Plan0 { - protected Pattern3 expression; - protected Func3 selector; - public Plan3(Pattern3 expression, Func3 selector) { - this.expression = expression; - this.selector = selector; - } - - @Override - public ActivePlan0 activate(Map externalSubscriptions, - final Observer observer, final Action1 deactivate) { - Action1 onError = Actions.onErrorFrom(observer); - - final JoinObserver1 firstJoinObserver = createObserver(externalSubscriptions, expression.first(), onError); - final JoinObserver1 secondJoinObserver = createObserver(externalSubscriptions, expression.second(), onError); - final JoinObserver1 thirdJoinObserver = createObserver(externalSubscriptions, expression.third(), onError); - - final AtomicReference> self = new AtomicReference>(); - - ActivePlan3 activePlan = new ActivePlan3(firstJoinObserver, secondJoinObserver, - thirdJoinObserver, new Action3() { - @Override - public void call(T1 t1, T2 t2, T3 t3) { - R result; - try { - result = selector.call(t1, t2, t3); - } catch (Throwable t) { - observer.onError(t); - return; - } - observer.onNext(result); - } - }, - new Action0() { - @Override - public void call() { - firstJoinObserver.removeActivePlan(self.get()); - secondJoinObserver.removeActivePlan(self.get()); - thirdJoinObserver.removeActivePlan(self.get()); - deactivate.call(self.get()); - } - }); - - self.set(activePlan); - - firstJoinObserver.addActivePlan(activePlan); - secondJoinObserver.addActivePlan(activePlan); - thirdJoinObserver.addActivePlan(activePlan); - - return activePlan; - } - -} +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ +package rx.joins; + +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; +import rx.Observer; +import static rx.joins.Plan0.createObserver; +import rx.util.functions.Action0; +import rx.util.functions.Action1; +import rx.util.functions.Action3; +import rx.util.functions.Actions; +import rx.util.functions.Func3; + +/** + * Represents an execution plan for join patterns. + */ +public class Plan3 extends Plan0 { + protected Pattern3 expression; + protected Func3 selector; + public Plan3(Pattern3 expression, Func3 selector) { + this.expression = expression; + this.selector = selector; + } + + @Override + public ActivePlan0 activate(Map externalSubscriptions, + final Observer observer, final Action1 deactivate) { + Action1 onError = Actions.onErrorFrom(observer); + + final JoinObserver1 firstJoinObserver = createObserver(externalSubscriptions, expression.first(), onError); + final JoinObserver1 secondJoinObserver = createObserver(externalSubscriptions, expression.second(), onError); + final JoinObserver1 thirdJoinObserver = createObserver(externalSubscriptions, expression.third(), onError); + + final AtomicReference> self = new AtomicReference>(); + + ActivePlan3 activePlan = new ActivePlan3(firstJoinObserver, secondJoinObserver, + thirdJoinObserver, new Action3() { + @Override + public void call(T1 t1, T2 t2, T3 t3) { + R result; + try { + result = selector.call(t1, t2, t3); + } catch (Throwable t) { + observer.onError(t); + return; + } + observer.onNext(result); + } + }, + new Action0() { + @Override + public void call() { + firstJoinObserver.removeActivePlan(self.get()); + secondJoinObserver.removeActivePlan(self.get()); + thirdJoinObserver.removeActivePlan(self.get()); + deactivate.call(self.get()); + } + }); + + self.set(activePlan); + + firstJoinObserver.addActivePlan(activePlan); + secondJoinObserver.addActivePlan(activePlan); + thirdJoinObserver.addActivePlan(activePlan); + + return activePlan; + } + +} diff --git a/rxjava-core/src/main/java/rx/operators/OperationJoinPatterns.java b/rxjava-core/src/main/java/rx/operators/OperationJoinPatterns.java index 6ff1c4346e..ffc304e12a 100644 --- a/rxjava-core/src/main/java/rx/operators/OperationJoinPatterns.java +++ b/rxjava-core/src/main/java/rx/operators/OperationJoinPatterns.java @@ -1,131 +1,131 @@ - /** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package rx.operators; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import rx.Observable; -import rx.Observable.OnSubscribeFunc; -import rx.Observer; -import rx.Subscription; -import rx.joins.ActivePlan0; -import rx.joins.JoinObserver; -import rx.joins.Pattern1; -import rx.joins.Pattern2; -import rx.joins.Plan0; -import rx.subjects.PublishSubject; -import rx.subscriptions.CompositeSubscription; -import rx.util.functions.Action1; -import rx.util.functions.Func1; -import rx.util.functions.Func2; - -/** - * Join patterns: And, Then, When. - */ -public class OperationJoinPatterns { - /** - * Creates a pattern that matches when both observable sequences have an available element. - */ - public static Pattern2 and(/* this */Observable left, Observable right) { - if (left == null) { - throw new NullPointerException("left"); - } - if (right == null) { - throw new NullPointerException("right"); - } - return new Pattern2(left, right); - } - /** - * Matches when the observable sequence has an available element and projects the element by invoking the selector function. - */ - public static Plan0 then(/* this */Observable source, Func1 selector) { - if (source == null) { - throw new NullPointerException("source"); - } - if (selector == null) { - throw new NullPointerException("selector"); - } - return new Pattern1(source).then(selector); - } - /** - * Joins together the results from several patterns. - */ - public static OnSubscribeFunc when(Plan0... plans) { - if (plans == null) { - throw new NullPointerException("plans"); - } - return when(Arrays.asList(plans)); - } - /** - * Joins together the results from several patterns. - */ - public static OnSubscribeFunc when(final Iterable> plans) { - if (plans == null) { - throw new NullPointerException("plans"); - } - return new OnSubscribeFunc() { - @Override - public Subscription onSubscribe(final Observer t1) { - final Map externalSubscriptions = new HashMap(); - final Object gate = new Object(); - final List activePlans = new ArrayList(); - - final Observer out = new Observer() { - @Override - public void onNext(R args) { - t1.onNext(args); - } - @Override - public void onError(Throwable e) { - for (JoinObserver po : externalSubscriptions.values()) { - po.unsubscribe(); - } - t1.onError(e); - } - @Override - public void onCompleted() { - t1.onCompleted(); - } - }; - - try { - for (Plan0 plan : plans) { - activePlans.add(plan.activate(externalSubscriptions, out, new Action1() { - @Override - public void call(ActivePlan0 activePlan) { - activePlans.remove(activePlan); - if (activePlans.isEmpty()) { - out.onCompleted(); - } - } - })); - } - } catch (Throwable t) { - return Observable.error(t).subscribe(t1); - } - CompositeSubscription group = new CompositeSubscription(); - for (JoinObserver jo : externalSubscriptions.values()) { - jo.subscribe(gate); - group.add(jo); - } - return group; - } - }; - } -} + /** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ +package rx.operators; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import rx.Observable; +import rx.Observable.OnSubscribeFunc; +import rx.Observer; +import rx.Subscription; +import rx.joins.ActivePlan0; +import rx.joins.JoinObserver; +import rx.joins.Pattern1; +import rx.joins.Pattern2; +import rx.joins.Plan0; +import rx.subjects.PublishSubject; +import rx.subscriptions.CompositeSubscription; +import rx.util.functions.Action1; +import rx.util.functions.Func1; +import rx.util.functions.Func2; + +/** + * Join patterns: And, Then, When. + */ +public class OperationJoinPatterns { + /** + * Creates a pattern that matches when both observable sequences have an available element. + */ + public static Pattern2 and(/* this */Observable left, Observable right) { + if (left == null) { + throw new NullPointerException("left"); + } + if (right == null) { + throw new NullPointerException("right"); + } + return new Pattern2(left, right); + } + /** + * Matches when the observable sequence has an available element and projects the element by invoking the selector function. + */ + public static Plan0 then(/* this */Observable source, Func1 selector) { + if (source == null) { + throw new NullPointerException("source"); + } + if (selector == null) { + throw new NullPointerException("selector"); + } + return new Pattern1(source).then(selector); + } + /** + * Joins together the results from several patterns. + */ + public static OnSubscribeFunc when(Plan0... plans) { + if (plans == null) { + throw new NullPointerException("plans"); + } + return when(Arrays.asList(plans)); + } + /** + * Joins together the results from several patterns. + */ + public static OnSubscribeFunc when(final Iterable> plans) { + if (plans == null) { + throw new NullPointerException("plans"); + } + return new OnSubscribeFunc() { + @Override + public Subscription onSubscribe(final Observer t1) { + final Map externalSubscriptions = new HashMap(); + final Object gate = new Object(); + final List activePlans = new ArrayList(); + + final Observer out = new Observer() { + @Override + public void onNext(R args) { + t1.onNext(args); + } + @Override + public void onError(Throwable e) { + for (JoinObserver po : externalSubscriptions.values()) { + po.unsubscribe(); + } + t1.onError(e); + } + @Override + public void onCompleted() { + t1.onCompleted(); + } + }; + + try { + for (Plan0 plan : plans) { + activePlans.add(plan.activate(externalSubscriptions, out, new Action1() { + @Override + public void call(ActivePlan0 activePlan) { + activePlans.remove(activePlan); + if (activePlans.isEmpty()) { + out.onCompleted(); + } + } + })); + } + } catch (Throwable t) { + return Observable.error(t).subscribe(t1); + } + CompositeSubscription group = new CompositeSubscription(); + for (JoinObserver jo : externalSubscriptions.values()) { + jo.subscribe(gate); + group.add(jo); + } + return group; + } + }; + } +} diff --git a/rxjava-core/src/main/java/rx/operators/OperationToMap.java b/rxjava-core/src/main/java/rx/operators/OperationToMap.java index c52b0244f2..754ff82d64 100644 --- a/rxjava-core/src/main/java/rx/operators/OperationToMap.java +++ b/rxjava-core/src/main/java/rx/operators/OperationToMap.java @@ -1,159 +1,159 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package rx.operators; - -import java.util.HashMap; -import java.util.Map; -import rx.Observable; -import rx.Observable.OnSubscribeFunc; -import rx.Observer; -import rx.Subscription; -import rx.subscriptions.Subscriptions; -import rx.util.functions.Func0; -import rx.util.functions.Func1; -import rx.util.functions.Functions; - -/** - * Maps the elements of the source observable into a java.util.Map instance and - * emits that once the source observable completes. - * - * @see Issue #96 - */ -public class OperationToMap { - /** - * ToMap with key selector, identity value selector and default HashMap factory. - */ - public static OnSubscribeFunc> toMap(Observable source, - Func1 keySelector) { - return new ToMap(source, keySelector, - Functions.identity(), new DefaultToMapFactory()); - } - - /** - * ToMap with key selector, value selector and default HashMap factory. - */ - public static OnSubscribeFunc> toMap(Observable source, - Func1 keySelector, - Func1 valueSelector) { - return new ToMap(source, keySelector, - valueSelector, new DefaultToMapFactory()); - } - - /** - * ToMap with key selector, value selector and custom Map factory. - */ - public static OnSubscribeFunc> toMap(Observable source, - Func1 keySelector, - Func1 valueSelector, - Func0> mapFactory) { - return new ToMap(source, keySelector, - valueSelector, mapFactory); - } - - /** The default map factory. */ - public static class DefaultToMapFactory implements Func0> { - @Override - public Map call() { - return new HashMap(); - } - } - /** - * Maps the elements of the source observable into a java.util.Map instance - * returned by the mapFactory function by using the keySelector and - * valueSelector. - * @param the source's value type - * @param the key type - * @param the value type - */ - public static class ToMap implements OnSubscribeFunc> { - /** The source. */ - private final Observable source; - /** Key extractor. */ - private final Func1 keySelector; - /** Value extractor. */ - private final Func1 valueSelector; - /** Map factory. */ - private final Func0> mapFactory; - public ToMap( - Observable source, - Func1 keySelector, - Func1 valueSelector, - Func0> mapFactory - ) { - this.source = source; - this.keySelector = keySelector; - this.valueSelector = valueSelector; - this.mapFactory = mapFactory; - - } - @Override - public Subscription onSubscribe(Observer> t1) { - Map map; - try { - map = mapFactory.call(); - } catch (Throwable t) { - t1.onError(t); - return Subscriptions.empty(); - } - return source.subscribe(new ToMapObserver( - t1, keySelector, valueSelector, map)); - } - /** - * Observer that collects the source values of T into - * a map. - */ - public static class ToMapObserver implements Observer { - /** The map. */ - Map map; - /** Key extractor. */ - private final Func1 keySelector; - /** Value extractor. */ - private final Func1 valueSelector; - /** The observer who is receiving the completed map. */ - private final Observer> t1; - - public ToMapObserver( - Observer> t1, - Func1 keySelector, - Func1 valueSelector, - Map map) { - this.map = map; - this.t1 = t1; - this.keySelector = keySelector; - this.valueSelector = valueSelector; - } - @Override - public void onNext(T args) { - K key = keySelector.call(args); - V value = valueSelector.call(args); - map.put(key, value); - } - @Override - public void onError(Throwable e) { - map = null; - t1.onError(e); - } - @Override - public void onCompleted() { - Map map0 = map; - map = null; - t1.onNext(map0); - t1.onCompleted(); - } - } - } -} +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package rx.operators; + +import java.util.HashMap; +import java.util.Map; +import rx.Observable; +import rx.Observable.OnSubscribeFunc; +import rx.Observer; +import rx.Subscription; +import rx.subscriptions.Subscriptions; +import rx.util.functions.Func0; +import rx.util.functions.Func1; +import rx.util.functions.Functions; + +/** + * Maps the elements of the source observable into a java.util.Map instance and + * emits that once the source observable completes. + * + * @see Issue #96 + */ +public class OperationToMap { + /** + * ToMap with key selector, identity value selector and default HashMap factory. + */ + public static OnSubscribeFunc> toMap(Observable source, + Func1 keySelector) { + return new ToMap(source, keySelector, + Functions.identity(), new DefaultToMapFactory()); + } + + /** + * ToMap with key selector, value selector and default HashMap factory. + */ + public static OnSubscribeFunc> toMap(Observable source, + Func1 keySelector, + Func1 valueSelector) { + return new ToMap(source, keySelector, + valueSelector, new DefaultToMapFactory()); + } + + /** + * ToMap with key selector, value selector and custom Map factory. + */ + public static OnSubscribeFunc> toMap(Observable source, + Func1 keySelector, + Func1 valueSelector, + Func0> mapFactory) { + return new ToMap(source, keySelector, + valueSelector, mapFactory); + } + + /** The default map factory. */ + public static class DefaultToMapFactory implements Func0> { + @Override + public Map call() { + return new HashMap(); + } + } + /** + * Maps the elements of the source observable into a java.util.Map instance + * returned by the mapFactory function by using the keySelector and + * valueSelector. + * @param the source's value type + * @param the key type + * @param the value type + */ + public static class ToMap implements OnSubscribeFunc> { + /** The source. */ + private final Observable source; + /** Key extractor. */ + private final Func1 keySelector; + /** Value extractor. */ + private final Func1 valueSelector; + /** Map factory. */ + private final Func0> mapFactory; + public ToMap( + Observable source, + Func1 keySelector, + Func1 valueSelector, + Func0> mapFactory + ) { + this.source = source; + this.keySelector = keySelector; + this.valueSelector = valueSelector; + this.mapFactory = mapFactory; + + } + @Override + public Subscription onSubscribe(Observer> t1) { + Map map; + try { + map = mapFactory.call(); + } catch (Throwable t) { + t1.onError(t); + return Subscriptions.empty(); + } + return source.subscribe(new ToMapObserver( + t1, keySelector, valueSelector, map)); + } + /** + * Observer that collects the source values of T into + * a map. + */ + public static class ToMapObserver implements Observer { + /** The map. */ + Map map; + /** Key extractor. */ + private final Func1 keySelector; + /** Value extractor. */ + private final Func1 valueSelector; + /** The observer who is receiving the completed map. */ + private final Observer> t1; + + public ToMapObserver( + Observer> t1, + Func1 keySelector, + Func1 valueSelector, + Map map) { + this.map = map; + this.t1 = t1; + this.keySelector = keySelector; + this.valueSelector = valueSelector; + } + @Override + public void onNext(T args) { + K key = keySelector.call(args); + V value = valueSelector.call(args); + map.put(key, value); + } + @Override + public void onError(Throwable e) { + map = null; + t1.onError(e); + } + @Override + public void onCompleted() { + Map map0 = map; + map = null; + t1.onNext(map0); + t1.onCompleted(); + } + } + } +} diff --git a/rxjava-core/src/main/java/rx/operators/OperationToMultimap.java b/rxjava-core/src/main/java/rx/operators/OperationToMultimap.java index e9cf8d9413..7210ee45e7 100644 --- a/rxjava-core/src/main/java/rx/operators/OperationToMultimap.java +++ b/rxjava-core/src/main/java/rx/operators/OperationToMultimap.java @@ -1,206 +1,206 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package rx.operators; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.Map; -import rx.Observable; -import rx.Observable.OnSubscribeFunc; -import rx.Observer; -import rx.Subscription; -import rx.subscriptions.Subscriptions; -import rx.util.functions.Func0; -import rx.util.functions.Func1; -import rx.util.functions.Functions; - -/** - * Maps the elements of the source observable into a multimap - * (Map<K, Collection<V>>) where each - * key entry has a collection of the source's values. - * - * @see Issue #97 - */ -public class OperationToMultimap { - /** - * ToMultimap with key selector, identitiy value selector, - * default HashMap factory and default ArrayList collection factory. - */ - public static OnSubscribeFunc>> toMultimap( - Observable source, - Func1 keySelector - ) { - return new ToMultimap( - source, keySelector, Functions.identity(), - new DefaultToMultimapFactory(), - new DefaultMultimapCollectionFactory() - ); - } - - /** - * ToMultimap with key selector, custom value selector, - * default HashMap factory and default ArrayList collection factory. - */ - public static OnSubscribeFunc>> toMultimap( - Observable source, - Func1 keySelector, - Func1 valueSelector - ) { - return new ToMultimap( - source, keySelector, valueSelector, - new DefaultToMultimapFactory(), - new DefaultMultimapCollectionFactory() - ); - } - /** - * ToMultimap with key selector, custom value selector, - * custom Map factory and default ArrayList collection factory. - */ - public static OnSubscribeFunc>> toMultimap( - Observable source, - Func1 keySelector, - Func1 valueSelector, - Func0>> mapFactory - ) { - return new ToMultimap( - source, keySelector, valueSelector, - mapFactory, - new DefaultMultimapCollectionFactory() - ); - } - /** - * ToMultimap with key selector, custom value selector, - * custom Map factory and custom collection factory. - */ - public static OnSubscribeFunc>> toMultimap( - Observable source, - Func1 keySelector, - Func1 valueSelector, - Func0>> mapFactory, - Func1> collectionFactory - ) { - return new ToMultimap( - source, keySelector, valueSelector, - mapFactory, - collectionFactory - ); - } - /** - * The default multimap factory returning a HashMap. - */ - public static class DefaultToMultimapFactory implements Func0>> { - @Override - public Map> call() { - return new HashMap>(); - } - } - /** - * The default collection factory for a key in the multimap returning - * an ArrayList independent of the key. - */ - public static class DefaultMultimapCollectionFactory - implements Func1> { - @Override - public Collection call(K t1) { - return new ArrayList(); - } - } - /** - * Maps the elements of the source observable int a multimap customized - * by various selectors and factories. - */ - public static class ToMultimap implements OnSubscribeFunc>> { - private final Observable source; - private final Func1 keySelector; - private final Func1 valueSelector; - private final Func0>> mapFactory; - private final Func1> collectionFactory; - public ToMultimap( - Observable source, - Func1 keySelector, - Func1 valueSelector, - Func0>> mapFactory, - Func1> collectionFactory - ) { - this.source = source; - this.keySelector = keySelector; - this.valueSelector = valueSelector; - this.mapFactory = mapFactory; - this.collectionFactory = collectionFactory; - } - @Override - public Subscription onSubscribe(Observer>> t1) { - Map> map; - try { - map = mapFactory.call(); - } catch (Throwable t) { - t1.onError(t); - return Subscriptions.empty(); - } - return source.subscribe(new ToMultimapObserver( - t1, keySelector, valueSelector, map, collectionFactory - )); - } - /** - * Observer that collects the source values of Ts into a multimap. - */ - public static class ToMultimapObserver implements Observer { - private final Func1 keySelector; - private final Func1 valueSelector; - private final Func1> collectionFactory; - private Map> map; - private Observer>> t1; - public ToMultimapObserver( - Observer>> t1, - Func1 keySelector, - Func1 valueSelector, - Map> map, - Func1> collectionFactory - ) { - this.t1 = t1; - this.keySelector = keySelector; - this.valueSelector = valueSelector; - this.collectionFactory = collectionFactory; - this.map = map; - } - @Override - public void onNext(T args) { - K key = keySelector.call(args); - V value = valueSelector.call(args); - Collection collection = map.get(key); - if (collection == null) { - collection = collectionFactory.call(key); - map.put(key, collection); - } - collection.add(value); - } - @Override - public void onError(Throwable e) { - map = null; - t1.onError(e); - } - @Override - public void onCompleted() { - Map> map0 = map; - map = null; - t1.onNext(map0); - t1.onCompleted(); - } - } - } -} +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package rx.operators; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import rx.Observable; +import rx.Observable.OnSubscribeFunc; +import rx.Observer; +import rx.Subscription; +import rx.subscriptions.Subscriptions; +import rx.util.functions.Func0; +import rx.util.functions.Func1; +import rx.util.functions.Functions; + +/** + * Maps the elements of the source observable into a multimap + * (Map<K, Collection<V>>) where each + * key entry has a collection of the source's values. + * + * @see Issue #97 + */ +public class OperationToMultimap { + /** + * ToMultimap with key selector, identitiy value selector, + * default HashMap factory and default ArrayList collection factory. + */ + public static OnSubscribeFunc>> toMultimap( + Observable source, + Func1 keySelector + ) { + return new ToMultimap( + source, keySelector, Functions.identity(), + new DefaultToMultimapFactory(), + new DefaultMultimapCollectionFactory() + ); + } + + /** + * ToMultimap with key selector, custom value selector, + * default HashMap factory and default ArrayList collection factory. + */ + public static OnSubscribeFunc>> toMultimap( + Observable source, + Func1 keySelector, + Func1 valueSelector + ) { + return new ToMultimap( + source, keySelector, valueSelector, + new DefaultToMultimapFactory(), + new DefaultMultimapCollectionFactory() + ); + } + /** + * ToMultimap with key selector, custom value selector, + * custom Map factory and default ArrayList collection factory. + */ + public static OnSubscribeFunc>> toMultimap( + Observable source, + Func1 keySelector, + Func1 valueSelector, + Func0>> mapFactory + ) { + return new ToMultimap( + source, keySelector, valueSelector, + mapFactory, + new DefaultMultimapCollectionFactory() + ); + } + /** + * ToMultimap with key selector, custom value selector, + * custom Map factory and custom collection factory. + */ + public static OnSubscribeFunc>> toMultimap( + Observable source, + Func1 keySelector, + Func1 valueSelector, + Func0>> mapFactory, + Func1> collectionFactory + ) { + return new ToMultimap( + source, keySelector, valueSelector, + mapFactory, + collectionFactory + ); + } + /** + * The default multimap factory returning a HashMap. + */ + public static class DefaultToMultimapFactory implements Func0>> { + @Override + public Map> call() { + return new HashMap>(); + } + } + /** + * The default collection factory for a key in the multimap returning + * an ArrayList independent of the key. + */ + public static class DefaultMultimapCollectionFactory + implements Func1> { + @Override + public Collection call(K t1) { + return new ArrayList(); + } + } + /** + * Maps the elements of the source observable int a multimap customized + * by various selectors and factories. + */ + public static class ToMultimap implements OnSubscribeFunc>> { + private final Observable source; + private final Func1 keySelector; + private final Func1 valueSelector; + private final Func0>> mapFactory; + private final Func1> collectionFactory; + public ToMultimap( + Observable source, + Func1 keySelector, + Func1 valueSelector, + Func0>> mapFactory, + Func1> collectionFactory + ) { + this.source = source; + this.keySelector = keySelector; + this.valueSelector = valueSelector; + this.mapFactory = mapFactory; + this.collectionFactory = collectionFactory; + } + @Override + public Subscription onSubscribe(Observer>> t1) { + Map> map; + try { + map = mapFactory.call(); + } catch (Throwable t) { + t1.onError(t); + return Subscriptions.empty(); + } + return source.subscribe(new ToMultimapObserver( + t1, keySelector, valueSelector, map, collectionFactory + )); + } + /** + * Observer that collects the source values of Ts into a multimap. + */ + public static class ToMultimapObserver implements Observer { + private final Func1 keySelector; + private final Func1 valueSelector; + private final Func1> collectionFactory; + private Map> map; + private Observer>> t1; + public ToMultimapObserver( + Observer>> t1, + Func1 keySelector, + Func1 valueSelector, + Map> map, + Func1> collectionFactory + ) { + this.t1 = t1; + this.keySelector = keySelector; + this.valueSelector = valueSelector; + this.collectionFactory = collectionFactory; + this.map = map; + } + @Override + public void onNext(T args) { + K key = keySelector.call(args); + V value = valueSelector.call(args); + Collection collection = map.get(key); + if (collection == null) { + collection = collectionFactory.call(key); + map.put(key, collection); + } + collection.add(value); + } + @Override + public void onError(Throwable e) { + map = null; + t1.onError(e); + } + @Override + public void onCompleted() { + Map> map0 = map; + map = null; + t1.onNext(map0); + t1.onCompleted(); + } + } + } +} diff --git a/rxjava-core/src/main/java/rx/subscriptions/SerialSubscription.java b/rxjava-core/src/main/java/rx/subscriptions/SerialSubscription.java index b206d37a7b..9aaa3e9a59 100644 --- a/rxjava-core/src/main/java/rx/subscriptions/SerialSubscription.java +++ b/rxjava-core/src/main/java/rx/subscriptions/SerialSubscription.java @@ -1,62 +1,62 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package rx.subscriptions; - -import static rx.subscriptions.Subscriptions.empty; - -import java.util.concurrent.atomic.AtomicReference; - -import rx.Subscription; - -/** - * Represents a subscription whose underlying subscription can be swapped for another subscription - * which causes the previous underlying subscription to be unsubscribed. - * - * @see Rx.Net equivalent SerialDisposable - */ -public class SerialSubscription implements Subscription { - private final AtomicReference reference = new AtomicReference(empty()); - - private static final Subscription UNSUBSCRIBED = new Subscription() { - @Override - public void unsubscribe() { - } - }; - - @Override - public void unsubscribe() { - setSubscription(UNSUBSCRIBED); - } - - public void setSubscription(final Subscription subscription) { - do { - final Subscription current = reference.get(); - if (current == UNSUBSCRIBED) { - subscription.unsubscribe(); - break; - } - if (reference.compareAndSet(current, subscription)) { - current.unsubscribe(); - break; - } - } while (true); - } - - public Subscription getSubscription() { - final Subscription subscription = reference.get(); - return subscription == UNSUBSCRIBED ? null : subscription; - } -} +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package rx.subscriptions; + +import static rx.subscriptions.Subscriptions.empty; + +import java.util.concurrent.atomic.AtomicReference; + +import rx.Subscription; + +/** + * Represents a subscription whose underlying subscription can be swapped for another subscription + * which causes the previous underlying subscription to be unsubscribed. + * + * @see Rx.Net equivalent SerialDisposable + */ +public class SerialSubscription implements Subscription { + private final AtomicReference reference = new AtomicReference(empty()); + + private static final Subscription UNSUBSCRIBED = new Subscription() { + @Override + public void unsubscribe() { + } + }; + + @Override + public void unsubscribe() { + setSubscription(UNSUBSCRIBED); + } + + public void setSubscription(final Subscription subscription) { + do { + final Subscription current = reference.get(); + if (current == UNSUBSCRIBED) { + subscription.unsubscribe(); + break; + } + if (reference.compareAndSet(current, subscription)) { + current.unsubscribe(); + break; + } + } while (true); + } + + public Subscription getSubscription() { + final Subscription subscription = reference.get(); + return subscription == UNSUBSCRIBED ? null : subscription; + } +} diff --git a/rxjava-core/src/main/java/rx/subscriptions/SingleAssignmentSubscription.java b/rxjava-core/src/main/java/rx/subscriptions/SingleAssignmentSubscription.java index 492c4e6dbf..ae79bc62a1 100644 --- a/rxjava-core/src/main/java/rx/subscriptions/SingleAssignmentSubscription.java +++ b/rxjava-core/src/main/java/rx/subscriptions/SingleAssignmentSubscription.java @@ -1,81 +1,81 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package rx.subscriptions; - -import java.util.concurrent.atomic.AtomicReference; -import rx.Subscription; - -/** - * A subscription that allows only a single resource to be assigned. - *

- * If this subscription is live, no other subscription may be set() and - * yields an {@link IllegalStateException}. - *

- * If the unsubscribe has been called, setting a new subscription will - * unsubscribe it immediately. - */ -public final class SingleAssignmentSubscription implements Subscription { - /** Holds the current resource. */ - private final AtomicReference current = new AtomicReference(); - /** Sentinel for the unsubscribed state. */ - private static final Subscription SENTINEL = new Subscription() { - @Override - public void unsubscribe() { - } - }; - /** - * Returns the current subscription or null if not yet set. - */ - public Subscription get() { - Subscription s = current.get(); - if (s == SENTINEL) { - return Subscriptions.empty(); - } - return s; - } - /** - * Sets a new subscription if not already set. - * @param s the new subscription - * @throws IllegalStateException if this subscription is live and contains - * another subscription. - */ - public void set(Subscription s) { - if (current.compareAndSet(null, s)) { - return; - } - if (current.get() != SENTINEL) { - throw new IllegalStateException("Subscription already set"); - } - if (s != null) { - s.unsubscribe(); - } - } - @Override - public void unsubscribe() { - Subscription old = current.getAndSet(SENTINEL); - if (old != null) { - old.unsubscribe(); - } - } - /** - * Test if this subscription is already unsubscribed. - */ - public boolean isUnsubscribed() { - return current.get() == SENTINEL; - } - -} +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package rx.subscriptions; + +import java.util.concurrent.atomic.AtomicReference; +import rx.Subscription; + +/** + * A subscription that allows only a single resource to be assigned. + *

+ * If this subscription is live, no other subscription may be set() and + * yields an {@link IllegalStateException}. + *

+ * If the unsubscribe has been called, setting a new subscription will + * unsubscribe it immediately. + */ +public final class SingleAssignmentSubscription implements Subscription { + /** Holds the current resource. */ + private final AtomicReference current = new AtomicReference(); + /** Sentinel for the unsubscribed state. */ + private static final Subscription SENTINEL = new Subscription() { + @Override + public void unsubscribe() { + } + }; + /** + * Returns the current subscription or null if not yet set. + */ + public Subscription get() { + Subscription s = current.get(); + if (s == SENTINEL) { + return Subscriptions.empty(); + } + return s; + } + /** + * Sets a new subscription if not already set. + * @param s the new subscription + * @throws IllegalStateException if this subscription is live and contains + * another subscription. + */ + public void set(Subscription s) { + if (current.compareAndSet(null, s)) { + return; + } + if (current.get() != SENTINEL) { + throw new IllegalStateException("Subscription already set"); + } + if (s != null) { + s.unsubscribe(); + } + } + @Override + public void unsubscribe() { + Subscription old = current.getAndSet(SENTINEL); + if (old != null) { + old.unsubscribe(); + } + } + /** + * Test if this subscription is already unsubscribed. + */ + public boolean isUnsubscribed() { + return current.get() == SENTINEL; + } + +} diff --git a/rxjava-core/src/main/java/rx/util/functions/Action4.java b/rxjava-core/src/main/java/rx/util/functions/Action4.java index f4126fcb95..3ab2ba1dc8 100644 --- a/rxjava-core/src/main/java/rx/util/functions/Action4.java +++ b/rxjava-core/src/main/java/rx/util/functions/Action4.java @@ -1,24 +1,24 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package rx.util.functions; - -/** - * A four-argument action. - */ -public interface Action4 extends Action { - void call(T1 t1, T2 t2, T3 t3, T4 t4); -} +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package rx.util.functions; + +/** + * A four-argument action. + */ +public interface Action4 extends Action { + void call(T1 t1, T2 t2, T3 t3, T4 t4); +} diff --git a/rxjava-core/src/main/java/rx/util/functions/Action5.java b/rxjava-core/src/main/java/rx/util/functions/Action5.java index 14d4bfa666..f15472464f 100644 --- a/rxjava-core/src/main/java/rx/util/functions/Action5.java +++ b/rxjava-core/src/main/java/rx/util/functions/Action5.java @@ -1,24 +1,24 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package rx.util.functions; - -/** - * A five-argument action. - */ -public interface Action5 extends Action { - void call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5); -} +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package rx.util.functions; + +/** + * A five-argument action. + */ +public interface Action5 extends Action { + void call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5); +} diff --git a/rxjava-core/src/main/java/rx/util/functions/Action6.java b/rxjava-core/src/main/java/rx/util/functions/Action6.java index 69cda2fe69..ff26b8cc69 100644 --- a/rxjava-core/src/main/java/rx/util/functions/Action6.java +++ b/rxjava-core/src/main/java/rx/util/functions/Action6.java @@ -1,24 +1,24 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package rx.util.functions; - -/** - * A six-argument action. - */ -public interface Action6 extends Action { - void call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6); -} +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package rx.util.functions; + +/** + * A six-argument action. + */ +public interface Action6 extends Action { + void call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6); +} diff --git a/rxjava-core/src/main/java/rx/util/functions/Action7.java b/rxjava-core/src/main/java/rx/util/functions/Action7.java index 1b7a5ade64..9af5268bbc 100644 --- a/rxjava-core/src/main/java/rx/util/functions/Action7.java +++ b/rxjava-core/src/main/java/rx/util/functions/Action7.java @@ -1,23 +1,23 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package rx.util.functions; - -/** - * A seven-argument action. - */ -public interface Action7 extends Action { - void call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7); -} +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package rx.util.functions; + +/** + * A seven-argument action. + */ +public interface Action7 extends Action { + void call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7); +} diff --git a/rxjava-core/src/main/java/rx/util/functions/Action8.java b/rxjava-core/src/main/java/rx/util/functions/Action8.java index 4ad0c7915d..175fa437b2 100644 --- a/rxjava-core/src/main/java/rx/util/functions/Action8.java +++ b/rxjava-core/src/main/java/rx/util/functions/Action8.java @@ -1,23 +1,23 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package rx.util.functions; - -/** - * An eight-argument action. - */ -public interface Action8 extends Action { - void call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8); -} +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package rx.util.functions; + +/** + * An eight-argument action. + */ +public interface Action8 extends Action { + void call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8); +} diff --git a/rxjava-core/src/main/java/rx/util/functions/Action9.java b/rxjava-core/src/main/java/rx/util/functions/Action9.java index 09a82f4949..b97536ee72 100644 --- a/rxjava-core/src/main/java/rx/util/functions/Action9.java +++ b/rxjava-core/src/main/java/rx/util/functions/Action9.java @@ -1,23 +1,23 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package rx.util.functions; - -/** - * A nine-argument action. - */ -public interface Action9 extends Action { - void call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9); -} +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package rx.util.functions; + +/** + * A nine-argument action. + */ +public interface Action9 extends Action { + void call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9); +} diff --git a/rxjava-core/src/main/java/rx/util/functions/ActionN.java b/rxjava-core/src/main/java/rx/util/functions/ActionN.java index b7ef56252f..95741e358e 100644 --- a/rxjava-core/src/main/java/rx/util/functions/ActionN.java +++ b/rxjava-core/src/main/java/rx/util/functions/ActionN.java @@ -1,23 +1,23 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package rx.util.functions; - -/** - * A vector-argument action. - */ -public interface ActionN extends Action { - void call(Object... args); -} +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package rx.util.functions; + +/** + * A vector-argument action. + */ +public interface ActionN extends Action { + void call(Object... args); +} diff --git a/rxjava-core/src/main/java/rx/util/functions/Actions.java b/rxjava-core/src/main/java/rx/util/functions/Actions.java index 117fa828b6..4fb017cb36 100644 --- a/rxjava-core/src/main/java/rx/util/functions/Actions.java +++ b/rxjava-core/src/main/java/rx/util/functions/Actions.java @@ -1,359 +1,359 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package rx.util.functions; - -import rx.Observer; -import rx.util.functions.Action0; -import rx.util.functions.Action1; - -/** - * Utility class for the Action interfaces. - */ -public final class Actions { - private Actions() { throw new IllegalStateException("No instances!"); } - /** - * Extracts a method reference to the observer's onNext method - * in the form of an Action1. - *

Java 8: observer::onNext

- * @param observer the observer to use - * @return an action which calls the observer's onNext method. - */ - public static Action1 onNextFrom(final Observer observer) { - return new Action1() { - @Override - public void call(T t1) { - observer.onNext(t1); - } - }; - } - /** - * Extracts a method reference to the observer's onError method - * in the form of an Action1. - *

Java 8: observer::onError

- * @param observer the observer to use - * @return an action which calls the observer's onError method. - */ - public static Action1 onErrorFrom(final Observer observer) { - return new Action1() { - @Override - public void call(Throwable t1) { - observer.onError(t1); - } - }; - } - /** - * Extracts a method reference to the observer's onCompleted method - * in the form of an Action0. - *

Java 8: observer::onCompleted

- * @param observer the observer to use - * @return an action which calls the observer's onCompleted method. - */ - public static Action0 onCompletedFrom(final Observer observer) { - return new Action0() { - @Override - public void call() { - observer.onCompleted(); - } - }; - } - /** - * Convert an action to a function which calls - * the action returns Void (null). - * @param action - * @return {@link Func0} - */ - public static Func0 toFunc(final Action0 action) { - return toFunc(action, (Void)null); - } - /** - * Convert an action to a function which calls - * the action returns Void (null). - * @param action - * @return {@link Func0} - */ - public static Func1 toFunc(final Action1 action) { - return toFunc(action, (Void)null); - } - /** - * Convert an action to a function which calls - * the action returns Void (null). - * @param action - * @return {@link Func0} - */ - public static Func2 toFunc(final Action2 action) { - return toFunc(action, (Void)null); - } - /** - * Convert an action to a function which calls - * the action returns Void (null). - * @param action - * @return {@link Func0} - */ - public static Func3 toFunc(final Action3 action) { - return toFunc(action, (Void)null); - } - /** - * Convert an action to a function which calls - * the action returns Void (null). - * @param action - * @return {@link Func0} - */ - public static Func4 toFunc(final Action4 action) { - return toFunc(action, (Void)null); - } - /** - * Convert an action to a function which calls - * the action returns Void (null). - * @param action - * @return {@link Func0} - */ - public static Func5 toFunc( - final Action5 action) { - return toFunc(action, (Void)null); - } - /** - * Convert an action to a function which calls - * the action returns Void (null). - * @param action - * @return {@link Func0} - */ - public static Func6 toFunc( - final Action6 action) { - return toFunc(action, (Void)null); - } - /** - * Convert an action to a function which calls - * the action returns Void (null). - * @param action - * @return {@link Func0} - */ - public static Func7 toFunc( - final Action7 action) { - return toFunc(action, (Void)null); - } - /** - * Convert an action to a function which calls - * the action returns Void (null). - * @param action - * @return {@link Func0} - */ - public static Func8 toFunc( - final Action8 action) { - return toFunc(action, (Void)null); - } - /** - * Convert an action to a function which calls - * the action returns Void (null). - * @param action - * @return {@link Func0} - */ - public static Func9 toFunc( - final Action9 action) { - return toFunc(action, (Void)null); - } - /** - * Convert an action to a function which calls - * the action returns Void (null). - * @param action - * @return {@link Func0} - */ - public static FuncN toFunc( - final ActionN action) { - return toFunc(action, (Void)null); - } - /** - * Convert an action to a function which calls - * the action returns the given result. - * @param action - * @param result - * @return {@link Func0} - */ - public static Func0 toFunc(final Action0 action, final R result) { - return new Func0() { - @Override - public R call() { - action.call(); - return result; - } - }; - } - /** - * Convert an action to a function which calls - * the action returns Void (null). - * @param action - * @param result - * @return {@link Func0} - */ - public static Func1 toFunc(final Action1 action, final R result) { - return new Func1() { - @Override - public R call(T1 t1) { - action.call(t1); - return result; - } - }; - } - /** - * Convert an action to a function which calls - * the action returns Void (null). - * @param action - * @param result - * @return {@link Func0} - */ - public static Func2 toFunc(final Action2 action, final R result) { - return new Func2() { - @Override - public R call(T1 t1, T2 t2) { - action.call(t1, t2); - return result; - } - }; - } - /** - * Convert an action to a function which calls - * the action returns Void (null). - * @param action - * @param result - * @return {@link Func0} - */ - public static Func3 toFunc(final Action3 action, final R result) { - return new Func3() { - @Override - public R call(T1 t1, T2 t2, T3 t3) { - action.call(t1, t2, t3); - return result; - } - }; - } - /** - * Convert an action to a function which calls - * the action returns Void (null). - * @param action - * @param result - * @return {@link Func0} - */ - public static Func4 toFunc(final Action4 action, final R result) { - return new Func4() { - @Override - public R call(T1 t1, T2 t2, T3 t3, T4 t4) { - action.call(t1, t2, t3, t4); - return result; - } - }; - } - /** - * Convert an action to a function which calls - * the action returns Void (null). - * @param action - * @param result - * @return {@link Func0} - */ - public static Func5 toFunc( - final Action5 action, final R result) { - return new Func5() { - @Override - public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5) { - action.call(t1, t2, t3, t4, t5); - return result; - } - }; - } - /** - * Convert an action to a function which calls - * the action returns Void (null). - * @param action - * @param result - * @return {@link Func0} - */ - public static Func6 toFunc( - final Action6 action, final R result) { - return new Func6() { - @Override - public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6) { - action.call(t1, t2, t3, t4, t5, t6); - return result; - } - }; - } - /** - * Convert an action to a function which calls - * the action returns Void (null). - * @param action - * @param result - * @return {@link Func0} - */ - public static Func7 toFunc( - final Action7 action, final R result) { - return new Func7() { - @Override - public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7) { - action.call(t1, t2, t3, t4, t5, t6, t7); - return result; - } - }; - } - /** - * Convert an action to a function which calls - * the action returns Void (null). - * @param action - * @param result - * @return {@link Func0} - */ - public static Func8 toFunc( - final Action8 action, final R result) { - return new Func8() { - @Override - public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8) { - action.call(t1, t2, t3, t4, t5, t6, t7, t8); - return result; - } - }; - } - /** - * Convert an action to a function which calls - * the action returns Void (null). - * @param action - * @param result - * @return {@link Func0} - */ - public static Func9 toFunc( - final Action9 action, final R result) { - return new Func9() { - @Override - public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9) { - action.call(t1, t2, t3, t4, t5, t6, t7, t8, t9); - return result; - } - }; - } - /** - * Convert an action to a function which calls - * the action returns Void (null). - * @param action - * @param result - * @return {@link Func0} - */ - public static FuncN toFunc( - final ActionN action, final R result) { - return new FuncN() { - @Override - public R call(Object... args) { - action.call(args); - return result; - } - }; - } -} +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ +package rx.util.functions; + +import rx.Observer; +import rx.util.functions.Action0; +import rx.util.functions.Action1; + +/** + * Utility class for the Action interfaces. + */ +public final class Actions { + private Actions() { throw new IllegalStateException("No instances!"); } + /** + * Extracts a method reference to the observer's onNext method + * in the form of an Action1. + *

Java 8: observer::onNext

+ * @param observer the observer to use + * @return an action which calls the observer's onNext method. + */ + public static Action1 onNextFrom(final Observer observer) { + return new Action1() { + @Override + public void call(T t1) { + observer.onNext(t1); + } + }; + } + /** + * Extracts a method reference to the observer's onError method + * in the form of an Action1. + *

Java 8: observer::onError

+ * @param observer the observer to use + * @return an action which calls the observer's onError method. + */ + public static Action1 onErrorFrom(final Observer observer) { + return new Action1() { + @Override + public void call(Throwable t1) { + observer.onError(t1); + } + }; + } + /** + * Extracts a method reference to the observer's onCompleted method + * in the form of an Action0. + *

Java 8: observer::onCompleted

+ * @param observer the observer to use + * @return an action which calls the observer's onCompleted method. + */ + public static Action0 onCompletedFrom(final Observer observer) { + return new Action0() { + @Override + public void call() { + observer.onCompleted(); + } + }; + } + /** + * Convert an action to a function which calls + * the action returns Void (null). + * @param action + * @return {@link Func0} + */ + public static Func0 toFunc(final Action0 action) { + return toFunc(action, (Void)null); + } + /** + * Convert an action to a function which calls + * the action returns Void (null). + * @param action + * @return {@link Func0} + */ + public static Func1 toFunc(final Action1 action) { + return toFunc(action, (Void)null); + } + /** + * Convert an action to a function which calls + * the action returns Void (null). + * @param action + * @return {@link Func0} + */ + public static Func2 toFunc(final Action2 action) { + return toFunc(action, (Void)null); + } + /** + * Convert an action to a function which calls + * the action returns Void (null). + * @param action + * @return {@link Func0} + */ + public static Func3 toFunc(final Action3 action) { + return toFunc(action, (Void)null); + } + /** + * Convert an action to a function which calls + * the action returns Void (null). + * @param action + * @return {@link Func0} + */ + public static Func4 toFunc(final Action4 action) { + return toFunc(action, (Void)null); + } + /** + * Convert an action to a function which calls + * the action returns Void (null). + * @param action + * @return {@link Func0} + */ + public static Func5 toFunc( + final Action5 action) { + return toFunc(action, (Void)null); + } + /** + * Convert an action to a function which calls + * the action returns Void (null). + * @param action + * @return {@link Func0} + */ + public static Func6 toFunc( + final Action6 action) { + return toFunc(action, (Void)null); + } + /** + * Convert an action to a function which calls + * the action returns Void (null). + * @param action + * @return {@link Func0} + */ + public static Func7 toFunc( + final Action7 action) { + return toFunc(action, (Void)null); + } + /** + * Convert an action to a function which calls + * the action returns Void (null). + * @param action + * @return {@link Func0} + */ + public static Func8 toFunc( + final Action8 action) { + return toFunc(action, (Void)null); + } + /** + * Convert an action to a function which calls + * the action returns Void (null). + * @param action + * @return {@link Func0} + */ + public static Func9 toFunc( + final Action9 action) { + return toFunc(action, (Void)null); + } + /** + * Convert an action to a function which calls + * the action returns Void (null). + * @param action + * @return {@link Func0} + */ + public static FuncN toFunc( + final ActionN action) { + return toFunc(action, (Void)null); + } + /** + * Convert an action to a function which calls + * the action returns the given result. + * @param action + * @param result + * @return {@link Func0} + */ + public static Func0 toFunc(final Action0 action, final R result) { + return new Func0() { + @Override + public R call() { + action.call(); + return result; + } + }; + } + /** + * Convert an action to a function which calls + * the action returns Void (null). + * @param action + * @param result + * @return {@link Func0} + */ + public static Func1 toFunc(final Action1 action, final R result) { + return new Func1() { + @Override + public R call(T1 t1) { + action.call(t1); + return result; + } + }; + } + /** + * Convert an action to a function which calls + * the action returns Void (null). + * @param action + * @param result + * @return {@link Func0} + */ + public static Func2 toFunc(final Action2 action, final R result) { + return new Func2() { + @Override + public R call(T1 t1, T2 t2) { + action.call(t1, t2); + return result; + } + }; + } + /** + * Convert an action to a function which calls + * the action returns Void (null). + * @param action + * @param result + * @return {@link Func0} + */ + public static Func3 toFunc(final Action3 action, final R result) { + return new Func3() { + @Override + public R call(T1 t1, T2 t2, T3 t3) { + action.call(t1, t2, t3); + return result; + } + }; + } + /** + * Convert an action to a function which calls + * the action returns Void (null). + * @param action + * @param result + * @return {@link Func0} + */ + public static Func4 toFunc(final Action4 action, final R result) { + return new Func4() { + @Override + public R call(T1 t1, T2 t2, T3 t3, T4 t4) { + action.call(t1, t2, t3, t4); + return result; + } + }; + } + /** + * Convert an action to a function which calls + * the action returns Void (null). + * @param action + * @param result + * @return {@link Func0} + */ + public static Func5 toFunc( + final Action5 action, final R result) { + return new Func5() { + @Override + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5) { + action.call(t1, t2, t3, t4, t5); + return result; + } + }; + } + /** + * Convert an action to a function which calls + * the action returns Void (null). + * @param action + * @param result + * @return {@link Func0} + */ + public static Func6 toFunc( + final Action6 action, final R result) { + return new Func6() { + @Override + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6) { + action.call(t1, t2, t3, t4, t5, t6); + return result; + } + }; + } + /** + * Convert an action to a function which calls + * the action returns Void (null). + * @param action + * @param result + * @return {@link Func0} + */ + public static Func7 toFunc( + final Action7 action, final R result) { + return new Func7() { + @Override + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7) { + action.call(t1, t2, t3, t4, t5, t6, t7); + return result; + } + }; + } + /** + * Convert an action to a function which calls + * the action returns Void (null). + * @param action + * @param result + * @return {@link Func0} + */ + public static Func8 toFunc( + final Action8 action, final R result) { + return new Func8() { + @Override + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8) { + action.call(t1, t2, t3, t4, t5, t6, t7, t8); + return result; + } + }; + } + /** + * Convert an action to a function which calls + * the action returns Void (null). + * @param action + * @param result + * @return {@link Func0} + */ + public static Func9 toFunc( + final Action9 action, final R result) { + return new Func9() { + @Override + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9) { + action.call(t1, t2, t3, t4, t5, t6, t7, t8, t9); + return result; + } + }; + } + /** + * Convert an action to a function which calls + * the action returns Void (null). + * @param action + * @param result + * @return {@link Func0} + */ + public static FuncN toFunc( + final ActionN action, final R result) { + return new FuncN() { + @Override + public R call(Object... args) { + action.call(args); + return result; + } + }; + } +} diff --git a/rxjava-core/src/main/java/rx/util/functions/Async.java b/rxjava-core/src/main/java/rx/util/functions/Async.java index dc4dd4dffe..f31ab3780c 100644 --- a/rxjava-core/src/main/java/rx/util/functions/Async.java +++ b/rxjava-core/src/main/java/rx/util/functions/Async.java @@ -1,957 +1,957 @@ - /** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package rx.util.functions; - -import rx.Observable; -import rx.Scheduler; -import rx.concurrency.ExecutorScheduler; -import rx.concurrency.Schedulers; -import rx.subjects.AsyncSubject; - -/** - * Utility methods to convert functions and actions into asynchronous - * operations through the Observable/Observer pattern. - */ -public final class Async { - private Async() { throw new IllegalStateException("No instances!"); } - /** - * {@link Scheduler} intended for asynchronous conversions. - *

- * Defaults to {@link #threadPoolForComputation()}. - * - * @return {@link ExecutorScheduler} for asynchronous conversion work. - */ - public static Scheduler threadPoolForAsyncConversions() { - return Schedulers.threadPoolForComputation(); - } - /** - * Convert a synchronous action call into an asynchronous function - * call through an Observable sequence. - * - * @param action the action to convert - * - * @return a function which returns an observable sequence which - * executes the {@code action} and emits {@code null}. - * - * @see MSDN: Observable.ToAsync - */ - public static Func0> toAsync(Action0 action) { - return toAsync(action, threadPoolForAsyncConversions()); - } - /** - * Convert a synchronous function call into an asynchronous function - * call through an Observable sequence. - * - * @param func the function to convert - * - * @return a function which returns an observable sequence which - * executes the {@code func} and emits its returned value. - * - * @see MSDN: Observable.ToAsync - */ - public static Func0> toAsync(Func0 func) { - return toAsync(func, threadPoolForAsyncConversions()); - } - /** - * Convert a synchronous action call into an asynchronous function - * call through an Observable sequence. - * - * @param action the action to convert - * - * @return a function which returns an observable sequence which - * executes the {@code action} and emits {@code null}. - * - * @see MSDN: Observable.ToAsync - */ - public static Func1> toAsync(Action1 action) { - return toAsync(action, threadPoolForAsyncConversions()); - } - /** - * Convert a synchronous function call into an asynchronous function - * call through an Observable sequence. - * - * @param func the function to convert - * - * @return a function which returns an observable sequence which - * executes the {@code func} and emits its returned value. - * - * @see MSDN: Observable.ToAsync - */ - public static Func1> toAsync(Func1 func) { - return toAsync(func, threadPoolForAsyncConversions()); - } - /** - * Convert a synchronous action call into an asynchronous function - * call through an Observable sequence. - * - * @param action the action to convert - * - * @return a function which returns an observable sequence which - * executes the {@code action} and emits {@code null}. - * - * @see MSDN: Observable.ToAsync - */ - public static Func2> toAsync(Action2 action) { - return toAsync(action, threadPoolForAsyncConversions()); - } - /** - * Convert a synchronous function call into an asynchronous function - * call through an Observable sequence. - * - * @param func the function to convert - * - * @return a function which returns an observable sequence which - * executes the {@code func} and emits its returned value. - * - * @see MSDN: Observable.ToAsync - */ - public static Func2> toAsync(Func2 func) { - return toAsync(func, threadPoolForAsyncConversions()); - } - /** - * Convert a synchronous action call into an asynchronous function - * call through an Observable sequence. - * - * @param action the action to convert - * - * @return a function which returns an observable sequence which - * executes the {@code action} and emits {@code null}. - * - * @see MSDN: Observable.ToAsync - */ - public static Func3> toAsync(Action3 action) { - return toAsync(action, threadPoolForAsyncConversions()); - } - /** - * Convert a synchronous function call into an asynchronous function - * call through an Observable sequence. - * - * @param func the function to convert - * - * @return a function which returns an observable sequence which - * executes the {@code func} and emits its returned value. - * - * @see MSDN: Observable.ToAsync - */ - public static Func3> toAsync(Func3 func) { - return toAsync(func, threadPoolForAsyncConversions()); - } - /** - * Convert a synchronous action call into an asynchronous function - * call through an Observable sequence. - * - * @param action the action to convert - * - * @return a function which returns an observable sequence which - * executes the {@code action} and emits {@code null}. - * - * @see MSDN: Observable.ToAsync - */ - public static Func4> toAsync(Action4 action) { - return toAsync(action, threadPoolForAsyncConversions()); - } - /** - * Convert a synchronous function call into an asynchronous function - * call through an Observable sequence. - * - * @param func the function to convert - * - * @return a function which returns an observable sequence which - * executes the {@code func} and emits its returned value. - * - * @see MSDN: Observable.ToAsync - */ - public static Func4> toAsync(Func4 func) { - return toAsync(func, threadPoolForAsyncConversions()); - } - /** - * Convert a synchronous action call into an asynchronous function - * call through an Observable sequence. - * - * @param action the action to convert - * - * @return a function which returns an observable sequence which - * executes the {@code action} and emits {@code null}. - * - * @see MSDN: Observable.ToAsync - */ - public static Func5> toAsync(Action5 action) { - return toAsync(action, threadPoolForAsyncConversions()); - } - /** - * Convert a synchronous function call into an asynchronous function - * call through an Observable sequence. - * - * @param func the function to convert - * - * @return a function which returns an observable sequence which - * executes the {@code func} and emits its returned value. - * - * @see MSDN: Observable.ToAsync - */ - public static Func5> toAsync(Func5 func) { - return toAsync(func, threadPoolForAsyncConversions()); - } - /** - * Convert a synchronous action call into an asynchronous function - * call through an Observable sequence. - * - * @param action the action to convert - * - * @return a function which returns an observable sequence which - * executes the {@code action} and emits {@code null}. - * - * @see MSDN: Observable.ToAsync - */ - public static Func6> toAsync(Action6 action) { - return toAsync(action, threadPoolForAsyncConversions()); - } - /** - * Convert a synchronous function call into an asynchronous function - * call through an Observable sequence. - * - * @param func the function to convert - * - * @return a function which returns an observable sequence which - * executes the {@code func} and emits its returned value. - * - * @see MSDN: Observable.ToAsync - */ - public static Func6> toAsync(Func6 func) { - return toAsync(func, threadPoolForAsyncConversions()); - } - /** - * Convert a synchronous action call into an asynchronous function - * call through an Observable sequence. - * - * @param action the action to convert - * - * @return a function which returns an observable sequence which - * executes the {@code action} and emits {@code null}. - * - * @see MSDN: Observable.ToAsync - */ - public static Func7> toAsync(Action7 action) { - return toAsync(action, threadPoolForAsyncConversions()); - } - /** - * Convert a synchronous function call into an asynchronous function - * call through an Observable sequence. - * - * @param func the function to convert - * - * @return a function which returns an observable sequence which - * executes the {@code func} and emits its returned value. - * - * @see MSDN: Observable.ToAsync - */ - public static Func7> toAsync(Func7 func) { - return toAsync(func, threadPoolForAsyncConversions()); - } - /** - * Convert a synchronous action call into an asynchronous function - * call through an Observable sequence. - * - * @param action the action to convert - * - * @return a function which returns an observable sequence which - * executes the {@code action} and emits {@code null}. - * - * @see MSDN: Observable.ToAsync - */ - public static Func8> toAsync(Action8 action) { - return toAsync(action, threadPoolForAsyncConversions()); - } - /** - * Convert a synchronous function call into an asynchronous function - * call through an Observable sequence. - * - * @param func the function to convert - * - * @return a function which returns an observable sequence which - * executes the {@code func} and emits its returned value. - * - * @see MSDN: Observable.ToAsync - */ - public static Func8> toAsync(Func8 func) { - return toAsync(func, threadPoolForAsyncConversions()); - } - /** - * Convert a synchronous action call into an asynchronous function - * call through an Observable sequence. - * - * @param action the action to convert - * - * @return a function which returns an observable sequence which - * executes the {@code action} and emits {@code null}. - * - * @see MSDN: Observable.ToAsync - */ - public static Func9> toAsync(Action9 action) { - return toAsync(action, threadPoolForAsyncConversions()); - } - /** - * Convert a synchronous function call into an asynchronous function - * call through an Observable sequence. - * - * @param func the function to convert - * - * @return a function which returns an observable sequence which - * executes the {@code func} and emits its returned value. - * - * @see MSDN: Observable.ToAsync - */ - public static Func9> toAsync(Func9 func) { - return toAsync(func, threadPoolForAsyncConversions()); - } - /** - * Convert a synchronous action call into an asynchronous function - * call through an Observable sequence. - * - * @param action the action to convert - * - * @return a function which returns an observable sequence which - * executes the {@code action} and emits {@code null}. - * - */ - public static FuncN> toAsync(ActionN action) { - return toAsync(action, threadPoolForAsyncConversions()); - } - /** - * Convert a synchronous function call into an asynchronous function - * call through an Observable sequence. - * - * @param func the function to convert - * - * @return a function which returns an observable sequence which - * executes the {@code func} and emits its returned value. - * - */ - public static FuncN> toAsync(FuncN func) { - return toAsync(func, threadPoolForAsyncConversions()); - } - /** - * Convert a synchronous action call into an asynchronous function - * call through an Observable sequence. - * - * @param action the action to convert - * @param scheduler the scheduler used to execute the {@code action} - * - * @return a function which returns an observable sequence which - * executes the {@code action} and emits {@code null}. - * - * @see MSDN: Observable.ToAsync - */ - public static Func0> toAsync(final Action0 action, final Scheduler scheduler) { - return toAsync(Actions.toFunc(action), scheduler); - } - /** - * Convert a synchronous function call into an asynchronous function - * call through an Observable sequence. - * - * @param func the function to convert - * @param scheduler the scheduler used to call the {@code func} - * - * @return a function which returns an observable sequence which - * executes the {@code func} and emits its returned value. - * - * @see MSDN: Observable.ToAsync - */ - public static Func0> toAsync(final Func0 func, final Scheduler scheduler) { - return new Func0>() { - @Override - public Observable call() { - final AsyncSubject subject = AsyncSubject.create(); - scheduler.schedule(new Action0() { - @Override - public void call() { - R result; - try { - result = func.call(); - } catch (Throwable t) { - subject.onError(t); - return; - } - subject.onNext(result); - subject.onCompleted(); - } - }); - return subject; - } - }; - } - /** - * Convert a synchronous action call into an asynchronous function - * call through an Observable sequence. - * - * @param action the action to convert - * @param scheduler the scheduler used to execute the {@code action} - * - * @return a function which returns an observable sequence which - * executes the {@code action} and emits {@code null}. - * - * @see MSDN: Observable.ToAsync - */ - public static Func1> toAsync(final Action1 action, final Scheduler scheduler) { - return toAsync(Actions.toFunc(action), scheduler); - } - /** - * Convert a synchronous function call into an asynchronous function - * call through an Observable sequence. - * - * @param func the function to convert - * @param scheduler the scheduler used to call the {@code func} - * - * @return a function which returns an observable sequence which - * executes the {@code func} and emits its returned value. - * - * @see MSDN: Observable.ToAsync - */ - public static Func1> toAsync(final Func1 func, final Scheduler scheduler) { - return new Func1>() { - @Override - public Observable call(final T1 t1) { - final AsyncSubject subject = AsyncSubject.create(); - scheduler.schedule(new Action0() { - @Override - public void call() { - R result; - try { - result = func.call(t1); - } catch (Throwable t) { - subject.onError(t); - return; - } - subject.onNext(result); - subject.onCompleted(); - } - }); - return subject; - } - }; - } - /** - * Convert a synchronous action call into an asynchronous function - * call through an Observable sequence. - * - * @param action the action to convert - * @param scheduler the scheduler used to execute the {@code action} - * - * @return a function which returns an observable sequence which - * executes the {@code action} and emits {@code null}. - * - * @see MSDN: Observable.ToAsync - */ - public static Func2> toAsync(final Action2 action, final Scheduler scheduler) { - return toAsync(Actions.toFunc(action), scheduler); - } - /** - * Convert a synchronous function call into an asynchronous function - * call through an Observable sequence. - * - * @param func the function to convert - * @param scheduler the scheduler used to call the {@code func} - * - * @return a function which returns an observable sequence which - * executes the {@code func} and emits its returned value. - * - * @see MSDN: Observable.ToAsync - */ - public static Func2> toAsync(final Func2 func, final Scheduler scheduler) { - return new Func2>() { - @Override - public Observable call(final T1 t1, final T2 t2) { - final AsyncSubject subject = AsyncSubject.create(); - scheduler.schedule(new Action0() { - @Override - public void call() { - R result; - try { - result = func.call(t1, t2); - } catch (Throwable t) { - subject.onError(t); - return; - } - subject.onNext(result); - subject.onCompleted(); - } - }); - return subject; - } - }; - } - /** - * Convert a synchronous action call into an asynchronous function - * call through an Observable sequence. - * - * @param action the action to convert - * @param scheduler the scheduler used to execute the {@code action} - * - * @return a function which returns an observable sequence which - * executes the {@code action} and emits {@code null}. - * - * @see MSDN: Observable.ToAsync - */ - public static Func3> toAsync(final Action3 action, final Scheduler scheduler) { - return toAsync(Actions.toFunc(action), scheduler); - } - /** - * Convert a synchronous function call into an asynchronous function - * call through an Observable sequence. - * - * @param func the function to convert - * @param scheduler the scheduler used to call the {@code func} - * - * @return a function which returns an observable sequence which - * executes the {@code func} and emits its returned value. - * - * @see MSDN: Observable.ToAsync - */ - public static Func3> toAsync(final Func3 func, final Scheduler scheduler) { - return new Func3>() { - @Override - public Observable call(final T1 t1, final T2 t2, final T3 t3) { - final AsyncSubject subject = AsyncSubject.create(); - scheduler.schedule(new Action0() { - @Override - public void call() { - R result; - try { - result = func.call(t1, t2, t3); - } catch (Throwable t) { - subject.onError(t); - return; - } - subject.onNext(result); - subject.onCompleted(); - } - }); - return subject; - } - }; - } - /** - * Convert a synchronous action call into an asynchronous function - * call through an Observable sequence. - * - * @param action the action to convert - * @param scheduler the scheduler used to execute the {@code action} - * - * @return a function which returns an observable sequence which - * executes the {@code action} and emits {@code null}. - * - * @see MSDN: Observable.ToAsync - */ - public static Func4> toAsync(final Action4 action, final Scheduler scheduler) { - return toAsync(Actions.toFunc(action), scheduler); - } - /** - * Convert a synchronous function call into an asynchronous function - * call through an Observable sequence. - * - * @param func the function to convert - * @param scheduler the scheduler used to call the {@code func} - * - * @return a function which returns an observable sequence which - * executes the {@code func} and emits its returned value. - * - * @see MSDN: Observable.ToAsync - */ - public static Func4> toAsync(final Func4 func, final Scheduler scheduler) { - return new Func4>() { - @Override - public Observable call(final T1 t1, final T2 t2, final T3 t3, final T4 t4) { - final AsyncSubject subject = AsyncSubject.create(); - scheduler.schedule(new Action0() { - @Override - public void call() { - R result; - try { - result = func.call(t1, t2, t3, t4); - } catch (Throwable t) { - subject.onError(t); - return; - } - subject.onNext(result); - subject.onCompleted(); - } - }); - return subject; - } - }; - } - /** - * Convert a synchronous action call into an asynchronous function - * call through an Observable sequence. - * - * @param action the action to convert - * @param scheduler the scheduler used to execute the {@code action} - * - * @return a function which returns an observable sequence which - * executes the {@code action} and emits {@code null}. - * - * @see MSDN: Observable.ToAsync - */ - public static Func5> toAsync(final Action5 action, final Scheduler scheduler) { - return toAsync(Actions.toFunc(action), scheduler); - } - /** - * Convert a synchronous function call into an asynchronous function - * call through an Observable sequence. - * - * @param func the function to convert - * @param scheduler the scheduler used to call the {@code func} - * - * @return a function which returns an observable sequence which - * executes the {@code func} and emits its returned value. - * - * @see MSDN: Observable.ToAsync - */ - public static Func5> toAsync(final Func5 func, final Scheduler scheduler) { - return new Func5>() { - @Override - public Observable call(final T1 t1, final T2 t2, final T3 t3, final T4 t4, final T5 t5) { - final AsyncSubject subject = AsyncSubject.create(); - scheduler.schedule(new Action0() { - @Override - public void call() { - R result; - try { - result = func.call(t1, t2, t3, t4, t5); - } catch (Throwable t) { - subject.onError(t); - return; - } - subject.onNext(result); - subject.onCompleted(); - } - }); - return subject; - } - }; - } - /** - * Convert a synchronous action call into an asynchronous function - * call through an Observable sequence. - * - * @param action the action to convert - * @param scheduler the scheduler used to execute the {@code action} - * - * @return a function which returns an observable sequence which - * executes the {@code action} and emits {@code null}. - * - * @see MSDN: Observable.ToAsync - */ - public static Func6> toAsync(final Action6 action, final Scheduler scheduler) { - return toAsync(Actions.toFunc(action), scheduler); - } - /** - * Convert a synchronous function call into an asynchronous function - * call through an Observable sequence. - * - * @param func the function to convert - * @param scheduler the scheduler used to call the {@code func} - * - * @return a function which returns an observable sequence which - * executes the {@code func} and emits its returned value. - * - * @see MSDN: Observable.ToAsync - */ - public static Func6> toAsync(final Func6 func, final Scheduler scheduler) { - return new Func6>() { - @Override - public Observable call(final T1 t1, final T2 t2, final T3 t3, final T4 t4, final T5 t5, final T6 t6) { - final AsyncSubject subject = AsyncSubject.create(); - scheduler.schedule(new Action0() { - @Override - public void call() { - R result; - try { - result = func.call(t1, t2, t3, t4, t5, t6); - } catch (Throwable t) { - subject.onError(t); - return; - } - subject.onNext(result); - subject.onCompleted(); - } - }); - return subject; - } - }; - } - /** - * Convert a synchronous action call into an asynchronous function - * call through an Observable sequence. - * - * @param action the action to convert - * @param scheduler the scheduler used to execute the {@code action} - * - * @return a function which returns an observable sequence which - * executes the {@code action} and emits {@code null}. - * - * @see MSDN: Observable.ToAsync - */ - public static Func7> toAsync(final Action7 action, final Scheduler scheduler) { - return toAsync(Actions.toFunc(action), scheduler); - } - /** - * Convert a synchronous function call into an asynchronous function - * call through an Observable sequence. - * - * @param func the function to convert - * @param scheduler the scheduler used to call the {@code func} - * - * @return a function which returns an observable sequence which - * executes the {@code func} and emits its returned value. - * - * @see MSDN: Observable.ToAsync - */ - public static Func7> toAsync(final Func7 func, final Scheduler scheduler) { - return new Func7>() { - @Override - public Observable call(final T1 t1, final T2 t2, final T3 t3, final T4 t4, final T5 t5, final T6 t6, final T7 t7) { - final AsyncSubject subject = AsyncSubject.create(); - scheduler.schedule(new Action0() { - @Override - public void call() { - R result; - try { - result = func.call(t1, t2, t3, t4, t5, t6, t7); - } catch (Throwable t) { - subject.onError(t); - return; - } - subject.onNext(result); - subject.onCompleted(); - } - }); - return subject; - } - }; - } - /** - * Convert a synchronous action call into an asynchronous function - * call through an Observable sequence. - * - * @param action the action to convert - * @param scheduler the scheduler used to execute the {@code action} - * - * @return a function which returns an observable sequence which - * executes the {@code action} and emits {@code null}. - * - * @see MSDN: Observable.ToAsync - */ - public static Func8> toAsync(final Action8 action, final Scheduler scheduler) { - return toAsync(Actions.toFunc(action), scheduler); - } - /** - * Convert a synchronous function call into an asynchronous function - * call through an Observable sequence. - * - * @param func the function to convert - * @param scheduler the scheduler used to call the {@code func} - * - * @return a function which returns an observable sequence which - * executes the {@code func} and emits its returned value. - * - * @see MSDN: Observable.ToAsync - */ - public static Func8> toAsync(final Func8 func, final Scheduler scheduler) { - return new Func8>() { - @Override - public Observable call(final T1 t1, final T2 t2, final T3 t3, final T4 t4, final T5 t5, final T6 t6, final T7 t7, final T8 t8) { - final AsyncSubject subject = AsyncSubject.create(); - scheduler.schedule(new Action0() { - @Override - public void call() { - R result; - try { - result = func.call(t1, t2, t3, t4, t5, t6, t7, t8); - } catch (Throwable t) { - subject.onError(t); - return; - } - subject.onNext(result); - subject.onCompleted(); - } - }); - return subject; - } - }; - } - /** - * Convert a synchronous action call into an asynchronous function - * call through an Observable sequence. - * - * @param action the action to convert - * @param scheduler the scheduler used to execute the {@code action} - * - * @return a function which returns an observable sequence which - * executes the {@code action} and emits {@code null}. - * - * @see MSDN: Observable.ToAsync - */ - public static Func9> toAsync(final Action9 action, final Scheduler scheduler) { - return toAsync(Actions.toFunc(action), scheduler); - } - /** - * Convert a synchronous function call into an asynchronous function - * call through an Observable sequence. - * - * @param func the function to convert - * @param scheduler the scheduler used to call the {@code func} - * - * @return a function which returns an observable sequence which - * executes the {@code func} and emits its returned value. - * - * @see MSDN: Observable.ToAsync - */ - public static Func9> toAsync(final Func9 func, final Scheduler scheduler) { - return new Func9>() { - @Override - public Observable call(final T1 t1, final T2 t2, final T3 t3, final T4 t4, final T5 t5, final T6 t6, final T7 t7, final T8 t8, final T9 t9) { - final AsyncSubject subject = AsyncSubject.create(); - scheduler.schedule(new Action0() { - @Override - public void call() { - R result; - try { - result = func.call(t1, t2, t3, t4, t5, t6, t7, t8, t9); - } catch (Throwable t) { - subject.onError(t); - return; - } - subject.onNext(result); - subject.onCompleted(); - } - }); - return subject; - } - }; - } - /** - * Convert a synchronous action call into an asynchronous function - * call through an Observable sequence. - * - * @param action the action to convert - * @param scheduler the scheduler used to execute the {@code action} - * - * @return a function which returns an observable sequence which - * executes the {@code action} and emits {@code null}. - * - */ - public static FuncN> toAsync(final ActionN action, final Scheduler scheduler) { - return toAsync(Actions.toFunc(action), scheduler); - } - /** - * Convert a synchronous function call into an asynchronous function - * call through an Observable sequence. - * - * @param func the function to convert - * @param scheduler the scheduler used to call the {@code func} - * - * @return a function which returns an observable sequence which - * executes the {@code func} and emits its returned value. - * - */ - public static FuncN> toAsync(final FuncN func, final Scheduler scheduler) { - return new FuncN>() { - @Override - public Observable call(final Object... args) { - final AsyncSubject subject = AsyncSubject.create(); - scheduler.schedule(new Action0() { - @Override - public void call() { - R result; - try { - result = func.call(args); - } catch (Throwable t) { - subject.onError(t); - return; - } - subject.onNext(result); - subject.onCompleted(); - } - }); - return subject; - } - }; - } - /** - * Convert a synchronous action call into an asynchronous function - * call through an Observable sequence. - *

- * Alias for toAsync(ActionN) intended for dynamic languages. - * - * @param action the action to convert - * - * @return a function which returns an observable sequence which - * executes the {@code action} and emits {@code null}. - * - */ - public static FuncN> asyncAction(final ActionN action) { - return toAsync(action); - } - /** - * Convert a synchronous action call into an asynchronous function - * call through an Observable sequence. - *

- * Alias for toAsync(ActionN, Scheduler) intended for dynamic languages. - * - * @param action the action to convert - * @param scheduler the scheduler used to execute the {@code action} - * - * @return a function which returns an observable sequence which - * executes the {@code action} and emits {@code null}. - * - */ - public static FuncN> asyncAction(final ActionN action, final Scheduler scheduler) { - return toAsync(action, scheduler); - } - /** - * Convert a synchronous function call into an asynchronous function - * call through an Observable sequence. - *

- * Alias for toAsync(FuncN) intended for dynamic languages. - * - * @param func the function to convert - * - * @return a function which returns an observable sequence which - * executes the {@code func} and emits its returned value. - * - */ - public static FuncN> asyncFunc(final FuncN func) { - return toAsync(func); - } - /** - * Convert a synchronous function call into an asynchronous function - * call through an Observable sequence. - *

- * Alias for toAsync(FuncN, Scheduler) intended for dynamic languages. - * - * @param func the function to convert - * @param scheduler the scheduler used to call the {@code func} - * - * @return a function which returns an observable sequence which - * executes the {@code func} and emits its returned value. - * - */ - public static FuncN> asyncFunc(final FuncN func, final Scheduler scheduler) { - return toAsync(func, scheduler); - } -} + /** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package rx.util.functions; + +import rx.Observable; +import rx.Scheduler; +import rx.concurrency.ExecutorScheduler; +import rx.concurrency.Schedulers; +import rx.subjects.AsyncSubject; + +/** + * Utility methods to convert functions and actions into asynchronous + * operations through the Observable/Observer pattern. + */ +public final class Async { + private Async() { throw new IllegalStateException("No instances!"); } + /** + * {@link Scheduler} intended for asynchronous conversions. + *

+ * Defaults to {@link #threadPoolForComputation()}. + * + * @return {@link ExecutorScheduler} for asynchronous conversion work. + */ + public static Scheduler threadPoolForAsyncConversions() { + return Schedulers.threadPoolForComputation(); + } + /** + * Convert a synchronous action call into an asynchronous function + * call through an Observable sequence. + * + * @param action the action to convert + * + * @return a function which returns an observable sequence which + * executes the {@code action} and emits {@code null}. + * + * @see MSDN: Observable.ToAsync + */ + public static Func0> toAsync(Action0 action) { + return toAsync(action, threadPoolForAsyncConversions()); + } + /** + * Convert a synchronous function call into an asynchronous function + * call through an Observable sequence. + * + * @param func the function to convert + * + * @return a function which returns an observable sequence which + * executes the {@code func} and emits its returned value. + * + * @see MSDN: Observable.ToAsync + */ + public static Func0> toAsync(Func0 func) { + return toAsync(func, threadPoolForAsyncConversions()); + } + /** + * Convert a synchronous action call into an asynchronous function + * call through an Observable sequence. + * + * @param action the action to convert + * + * @return a function which returns an observable sequence which + * executes the {@code action} and emits {@code null}. + * + * @see MSDN: Observable.ToAsync + */ + public static Func1> toAsync(Action1 action) { + return toAsync(action, threadPoolForAsyncConversions()); + } + /** + * Convert a synchronous function call into an asynchronous function + * call through an Observable sequence. + * + * @param func the function to convert + * + * @return a function which returns an observable sequence which + * executes the {@code func} and emits its returned value. + * + * @see MSDN: Observable.ToAsync + */ + public static Func1> toAsync(Func1 func) { + return toAsync(func, threadPoolForAsyncConversions()); + } + /** + * Convert a synchronous action call into an asynchronous function + * call through an Observable sequence. + * + * @param action the action to convert + * + * @return a function which returns an observable sequence which + * executes the {@code action} and emits {@code null}. + * + * @see MSDN: Observable.ToAsync + */ + public static Func2> toAsync(Action2 action) { + return toAsync(action, threadPoolForAsyncConversions()); + } + /** + * Convert a synchronous function call into an asynchronous function + * call through an Observable sequence. + * + * @param func the function to convert + * + * @return a function which returns an observable sequence which + * executes the {@code func} and emits its returned value. + * + * @see MSDN: Observable.ToAsync + */ + public static Func2> toAsync(Func2 func) { + return toAsync(func, threadPoolForAsyncConversions()); + } + /** + * Convert a synchronous action call into an asynchronous function + * call through an Observable sequence. + * + * @param action the action to convert + * + * @return a function which returns an observable sequence which + * executes the {@code action} and emits {@code null}. + * + * @see MSDN: Observable.ToAsync + */ + public static Func3> toAsync(Action3 action) { + return toAsync(action, threadPoolForAsyncConversions()); + } + /** + * Convert a synchronous function call into an asynchronous function + * call through an Observable sequence. + * + * @param func the function to convert + * + * @return a function which returns an observable sequence which + * executes the {@code func} and emits its returned value. + * + * @see MSDN: Observable.ToAsync + */ + public static Func3> toAsync(Func3 func) { + return toAsync(func, threadPoolForAsyncConversions()); + } + /** + * Convert a synchronous action call into an asynchronous function + * call through an Observable sequence. + * + * @param action the action to convert + * + * @return a function which returns an observable sequence which + * executes the {@code action} and emits {@code null}. + * + * @see MSDN: Observable.ToAsync + */ + public static Func4> toAsync(Action4 action) { + return toAsync(action, threadPoolForAsyncConversions()); + } + /** + * Convert a synchronous function call into an asynchronous function + * call through an Observable sequence. + * + * @param func the function to convert + * + * @return a function which returns an observable sequence which + * executes the {@code func} and emits its returned value. + * + * @see MSDN: Observable.ToAsync + */ + public static Func4> toAsync(Func4 func) { + return toAsync(func, threadPoolForAsyncConversions()); + } + /** + * Convert a synchronous action call into an asynchronous function + * call through an Observable sequence. + * + * @param action the action to convert + * + * @return a function which returns an observable sequence which + * executes the {@code action} and emits {@code null}. + * + * @see MSDN: Observable.ToAsync + */ + public static Func5> toAsync(Action5 action) { + return toAsync(action, threadPoolForAsyncConversions()); + } + /** + * Convert a synchronous function call into an asynchronous function + * call through an Observable sequence. + * + * @param func the function to convert + * + * @return a function which returns an observable sequence which + * executes the {@code func} and emits its returned value. + * + * @see MSDN: Observable.ToAsync + */ + public static Func5> toAsync(Func5 func) { + return toAsync(func, threadPoolForAsyncConversions()); + } + /** + * Convert a synchronous action call into an asynchronous function + * call through an Observable sequence. + * + * @param action the action to convert + * + * @return a function which returns an observable sequence which + * executes the {@code action} and emits {@code null}. + * + * @see MSDN: Observable.ToAsync + */ + public static Func6> toAsync(Action6 action) { + return toAsync(action, threadPoolForAsyncConversions()); + } + /** + * Convert a synchronous function call into an asynchronous function + * call through an Observable sequence. + * + * @param func the function to convert + * + * @return a function which returns an observable sequence which + * executes the {@code func} and emits its returned value. + * + * @see MSDN: Observable.ToAsync + */ + public static Func6> toAsync(Func6 func) { + return toAsync(func, threadPoolForAsyncConversions()); + } + /** + * Convert a synchronous action call into an asynchronous function + * call through an Observable sequence. + * + * @param action the action to convert + * + * @return a function which returns an observable sequence which + * executes the {@code action} and emits {@code null}. + * + * @see MSDN: Observable.ToAsync + */ + public static Func7> toAsync(Action7 action) { + return toAsync(action, threadPoolForAsyncConversions()); + } + /** + * Convert a synchronous function call into an asynchronous function + * call through an Observable sequence. + * + * @param func the function to convert + * + * @return a function which returns an observable sequence which + * executes the {@code func} and emits its returned value. + * + * @see MSDN: Observable.ToAsync + */ + public static Func7> toAsync(Func7 func) { + return toAsync(func, threadPoolForAsyncConversions()); + } + /** + * Convert a synchronous action call into an asynchronous function + * call through an Observable sequence. + * + * @param action the action to convert + * + * @return a function which returns an observable sequence which + * executes the {@code action} and emits {@code null}. + * + * @see MSDN: Observable.ToAsync + */ + public static Func8> toAsync(Action8 action) { + return toAsync(action, threadPoolForAsyncConversions()); + } + /** + * Convert a synchronous function call into an asynchronous function + * call through an Observable sequence. + * + * @param func the function to convert + * + * @return a function which returns an observable sequence which + * executes the {@code func} and emits its returned value. + * + * @see MSDN: Observable.ToAsync + */ + public static Func8> toAsync(Func8 func) { + return toAsync(func, threadPoolForAsyncConversions()); + } + /** + * Convert a synchronous action call into an asynchronous function + * call through an Observable sequence. + * + * @param action the action to convert + * + * @return a function which returns an observable sequence which + * executes the {@code action} and emits {@code null}. + * + * @see MSDN: Observable.ToAsync + */ + public static Func9> toAsync(Action9 action) { + return toAsync(action, threadPoolForAsyncConversions()); + } + /** + * Convert a synchronous function call into an asynchronous function + * call through an Observable sequence. + * + * @param func the function to convert + * + * @return a function which returns an observable sequence which + * executes the {@code func} and emits its returned value. + * + * @see MSDN: Observable.ToAsync + */ + public static Func9> toAsync(Func9 func) { + return toAsync(func, threadPoolForAsyncConversions()); + } + /** + * Convert a synchronous action call into an asynchronous function + * call through an Observable sequence. + * + * @param action the action to convert + * + * @return a function which returns an observable sequence which + * executes the {@code action} and emits {@code null}. + * + */ + public static FuncN> toAsync(ActionN action) { + return toAsync(action, threadPoolForAsyncConversions()); + } + /** + * Convert a synchronous function call into an asynchronous function + * call through an Observable sequence. + * + * @param func the function to convert + * + * @return a function which returns an observable sequence which + * executes the {@code func} and emits its returned value. + * + */ + public static FuncN> toAsync(FuncN func) { + return toAsync(func, threadPoolForAsyncConversions()); + } + /** + * Convert a synchronous action call into an asynchronous function + * call through an Observable sequence. + * + * @param action the action to convert + * @param scheduler the scheduler used to execute the {@code action} + * + * @return a function which returns an observable sequence which + * executes the {@code action} and emits {@code null}. + * + * @see MSDN: Observable.ToAsync + */ + public static Func0> toAsync(final Action0 action, final Scheduler scheduler) { + return toAsync(Actions.toFunc(action), scheduler); + } + /** + * Convert a synchronous function call into an asynchronous function + * call through an Observable sequence. + * + * @param func the function to convert + * @param scheduler the scheduler used to call the {@code func} + * + * @return a function which returns an observable sequence which + * executes the {@code func} and emits its returned value. + * + * @see MSDN: Observable.ToAsync + */ + public static Func0> toAsync(final Func0 func, final Scheduler scheduler) { + return new Func0>() { + @Override + public Observable call() { + final AsyncSubject subject = AsyncSubject.create(); + scheduler.schedule(new Action0() { + @Override + public void call() { + R result; + try { + result = func.call(); + } catch (Throwable t) { + subject.onError(t); + return; + } + subject.onNext(result); + subject.onCompleted(); + } + }); + return subject; + } + }; + } + /** + * Convert a synchronous action call into an asynchronous function + * call through an Observable sequence. + * + * @param action the action to convert + * @param scheduler the scheduler used to execute the {@code action} + * + * @return a function which returns an observable sequence which + * executes the {@code action} and emits {@code null}. + * + * @see MSDN: Observable.ToAsync + */ + public static Func1> toAsync(final Action1 action, final Scheduler scheduler) { + return toAsync(Actions.toFunc(action), scheduler); + } + /** + * Convert a synchronous function call into an asynchronous function + * call through an Observable sequence. + * + * @param func the function to convert + * @param scheduler the scheduler used to call the {@code func} + * + * @return a function which returns an observable sequence which + * executes the {@code func} and emits its returned value. + * + * @see MSDN: Observable.ToAsync + */ + public static Func1> toAsync(final Func1 func, final Scheduler scheduler) { + return new Func1>() { + @Override + public Observable call(final T1 t1) { + final AsyncSubject subject = AsyncSubject.create(); + scheduler.schedule(new Action0() { + @Override + public void call() { + R result; + try { + result = func.call(t1); + } catch (Throwable t) { + subject.onError(t); + return; + } + subject.onNext(result); + subject.onCompleted(); + } + }); + return subject; + } + }; + } + /** + * Convert a synchronous action call into an asynchronous function + * call through an Observable sequence. + * + * @param action the action to convert + * @param scheduler the scheduler used to execute the {@code action} + * + * @return a function which returns an observable sequence which + * executes the {@code action} and emits {@code null}. + * + * @see MSDN: Observable.ToAsync + */ + public static Func2> toAsync(final Action2 action, final Scheduler scheduler) { + return toAsync(Actions.toFunc(action), scheduler); + } + /** + * Convert a synchronous function call into an asynchronous function + * call through an Observable sequence. + * + * @param func the function to convert + * @param scheduler the scheduler used to call the {@code func} + * + * @return a function which returns an observable sequence which + * executes the {@code func} and emits its returned value. + * + * @see MSDN: Observable.ToAsync + */ + public static Func2> toAsync(final Func2 func, final Scheduler scheduler) { + return new Func2>() { + @Override + public Observable call(final T1 t1, final T2 t2) { + final AsyncSubject subject = AsyncSubject.create(); + scheduler.schedule(new Action0() { + @Override + public void call() { + R result; + try { + result = func.call(t1, t2); + } catch (Throwable t) { + subject.onError(t); + return; + } + subject.onNext(result); + subject.onCompleted(); + } + }); + return subject; + } + }; + } + /** + * Convert a synchronous action call into an asynchronous function + * call through an Observable sequence. + * + * @param action the action to convert + * @param scheduler the scheduler used to execute the {@code action} + * + * @return a function which returns an observable sequence which + * executes the {@code action} and emits {@code null}. + * + * @see MSDN: Observable.ToAsync + */ + public static Func3> toAsync(final Action3 action, final Scheduler scheduler) { + return toAsync(Actions.toFunc(action), scheduler); + } + /** + * Convert a synchronous function call into an asynchronous function + * call through an Observable sequence. + * + * @param func the function to convert + * @param scheduler the scheduler used to call the {@code func} + * + * @return a function which returns an observable sequence which + * executes the {@code func} and emits its returned value. + * + * @see MSDN: Observable.ToAsync + */ + public static Func3> toAsync(final Func3 func, final Scheduler scheduler) { + return new Func3>() { + @Override + public Observable call(final T1 t1, final T2 t2, final T3 t3) { + final AsyncSubject subject = AsyncSubject.create(); + scheduler.schedule(new Action0() { + @Override + public void call() { + R result; + try { + result = func.call(t1, t2, t3); + } catch (Throwable t) { + subject.onError(t); + return; + } + subject.onNext(result); + subject.onCompleted(); + } + }); + return subject; + } + }; + } + /** + * Convert a synchronous action call into an asynchronous function + * call through an Observable sequence. + * + * @param action the action to convert + * @param scheduler the scheduler used to execute the {@code action} + * + * @return a function which returns an observable sequence which + * executes the {@code action} and emits {@code null}. + * + * @see MSDN: Observable.ToAsync + */ + public static Func4> toAsync(final Action4 action, final Scheduler scheduler) { + return toAsync(Actions.toFunc(action), scheduler); + } + /** + * Convert a synchronous function call into an asynchronous function + * call through an Observable sequence. + * + * @param func the function to convert + * @param scheduler the scheduler used to call the {@code func} + * + * @return a function which returns an observable sequence which + * executes the {@code func} and emits its returned value. + * + * @see MSDN: Observable.ToAsync + */ + public static Func4> toAsync(final Func4 func, final Scheduler scheduler) { + return new Func4>() { + @Override + public Observable call(final T1 t1, final T2 t2, final T3 t3, final T4 t4) { + final AsyncSubject subject = AsyncSubject.create(); + scheduler.schedule(new Action0() { + @Override + public void call() { + R result; + try { + result = func.call(t1, t2, t3, t4); + } catch (Throwable t) { + subject.onError(t); + return; + } + subject.onNext(result); + subject.onCompleted(); + } + }); + return subject; + } + }; + } + /** + * Convert a synchronous action call into an asynchronous function + * call through an Observable sequence. + * + * @param action the action to convert + * @param scheduler the scheduler used to execute the {@code action} + * + * @return a function which returns an observable sequence which + * executes the {@code action} and emits {@code null}. + * + * @see MSDN: Observable.ToAsync + */ + public static Func5> toAsync(final Action5 action, final Scheduler scheduler) { + return toAsync(Actions.toFunc(action), scheduler); + } + /** + * Convert a synchronous function call into an asynchronous function + * call through an Observable sequence. + * + * @param func the function to convert + * @param scheduler the scheduler used to call the {@code func} + * + * @return a function which returns an observable sequence which + * executes the {@code func} and emits its returned value. + * + * @see MSDN: Observable.ToAsync + */ + public static Func5> toAsync(final Func5 func, final Scheduler scheduler) { + return new Func5>() { + @Override + public Observable call(final T1 t1, final T2 t2, final T3 t3, final T4 t4, final T5 t5) { + final AsyncSubject subject = AsyncSubject.create(); + scheduler.schedule(new Action0() { + @Override + public void call() { + R result; + try { + result = func.call(t1, t2, t3, t4, t5); + } catch (Throwable t) { + subject.onError(t); + return; + } + subject.onNext(result); + subject.onCompleted(); + } + }); + return subject; + } + }; + } + /** + * Convert a synchronous action call into an asynchronous function + * call through an Observable sequence. + * + * @param action the action to convert + * @param scheduler the scheduler used to execute the {@code action} + * + * @return a function which returns an observable sequence which + * executes the {@code action} and emits {@code null}. + * + * @see MSDN: Observable.ToAsync + */ + public static Func6> toAsync(final Action6 action, final Scheduler scheduler) { + return toAsync(Actions.toFunc(action), scheduler); + } + /** + * Convert a synchronous function call into an asynchronous function + * call through an Observable sequence. + * + * @param func the function to convert + * @param scheduler the scheduler used to call the {@code func} + * + * @return a function which returns an observable sequence which + * executes the {@code func} and emits its returned value. + * + * @see MSDN: Observable.ToAsync + */ + public static Func6> toAsync(final Func6 func, final Scheduler scheduler) { + return new Func6>() { + @Override + public Observable call(final T1 t1, final T2 t2, final T3 t3, final T4 t4, final T5 t5, final T6 t6) { + final AsyncSubject subject = AsyncSubject.create(); + scheduler.schedule(new Action0() { + @Override + public void call() { + R result; + try { + result = func.call(t1, t2, t3, t4, t5, t6); + } catch (Throwable t) { + subject.onError(t); + return; + } + subject.onNext(result); + subject.onCompleted(); + } + }); + return subject; + } + }; + } + /** + * Convert a synchronous action call into an asynchronous function + * call through an Observable sequence. + * + * @param action the action to convert + * @param scheduler the scheduler used to execute the {@code action} + * + * @return a function which returns an observable sequence which + * executes the {@code action} and emits {@code null}. + * + * @see MSDN: Observable.ToAsync + */ + public static Func7> toAsync(final Action7 action, final Scheduler scheduler) { + return toAsync(Actions.toFunc(action), scheduler); + } + /** + * Convert a synchronous function call into an asynchronous function + * call through an Observable sequence. + * + * @param func the function to convert + * @param scheduler the scheduler used to call the {@code func} + * + * @return a function which returns an observable sequence which + * executes the {@code func} and emits its returned value. + * + * @see MSDN: Observable.ToAsync + */ + public static Func7> toAsync(final Func7 func, final Scheduler scheduler) { + return new Func7>() { + @Override + public Observable call(final T1 t1, final T2 t2, final T3 t3, final T4 t4, final T5 t5, final T6 t6, final T7 t7) { + final AsyncSubject subject = AsyncSubject.create(); + scheduler.schedule(new Action0() { + @Override + public void call() { + R result; + try { + result = func.call(t1, t2, t3, t4, t5, t6, t7); + } catch (Throwable t) { + subject.onError(t); + return; + } + subject.onNext(result); + subject.onCompleted(); + } + }); + return subject; + } + }; + } + /** + * Convert a synchronous action call into an asynchronous function + * call through an Observable sequence. + * + * @param action the action to convert + * @param scheduler the scheduler used to execute the {@code action} + * + * @return a function which returns an observable sequence which + * executes the {@code action} and emits {@code null}. + * + * @see MSDN: Observable.ToAsync + */ + public static Func8> toAsync(final Action8 action, final Scheduler scheduler) { + return toAsync(Actions.toFunc(action), scheduler); + } + /** + * Convert a synchronous function call into an asynchronous function + * call through an Observable sequence. + * + * @param func the function to convert + * @param scheduler the scheduler used to call the {@code func} + * + * @return a function which returns an observable sequence which + * executes the {@code func} and emits its returned value. + * + * @see MSDN: Observable.ToAsync + */ + public static Func8> toAsync(final Func8 func, final Scheduler scheduler) { + return new Func8>() { + @Override + public Observable call(final T1 t1, final T2 t2, final T3 t3, final T4 t4, final T5 t5, final T6 t6, final T7 t7, final T8 t8) { + final AsyncSubject subject = AsyncSubject.create(); + scheduler.schedule(new Action0() { + @Override + public void call() { + R result; + try { + result = func.call(t1, t2, t3, t4, t5, t6, t7, t8); + } catch (Throwable t) { + subject.onError(t); + return; + } + subject.onNext(result); + subject.onCompleted(); + } + }); + return subject; + } + }; + } + /** + * Convert a synchronous action call into an asynchronous function + * call through an Observable sequence. + * + * @param action the action to convert + * @param scheduler the scheduler used to execute the {@code action} + * + * @return a function which returns an observable sequence which + * executes the {@code action} and emits {@code null}. + * + * @see MSDN: Observable.ToAsync + */ + public static Func9> toAsync(final Action9 action, final Scheduler scheduler) { + return toAsync(Actions.toFunc(action), scheduler); + } + /** + * Convert a synchronous function call into an asynchronous function + * call through an Observable sequence. + * + * @param func the function to convert + * @param scheduler the scheduler used to call the {@code func} + * + * @return a function which returns an observable sequence which + * executes the {@code func} and emits its returned value. + * + * @see MSDN: Observable.ToAsync + */ + public static Func9> toAsync(final Func9 func, final Scheduler scheduler) { + return new Func9>() { + @Override + public Observable call(final T1 t1, final T2 t2, final T3 t3, final T4 t4, final T5 t5, final T6 t6, final T7 t7, final T8 t8, final T9 t9) { + final AsyncSubject subject = AsyncSubject.create(); + scheduler.schedule(new Action0() { + @Override + public void call() { + R result; + try { + result = func.call(t1, t2, t3, t4, t5, t6, t7, t8, t9); + } catch (Throwable t) { + subject.onError(t); + return; + } + subject.onNext(result); + subject.onCompleted(); + } + }); + return subject; + } + }; + } + /** + * Convert a synchronous action call into an asynchronous function + * call through an Observable sequence. + * + * @param action the action to convert + * @param scheduler the scheduler used to execute the {@code action} + * + * @return a function which returns an observable sequence which + * executes the {@code action} and emits {@code null}. + * + */ + public static FuncN> toAsync(final ActionN action, final Scheduler scheduler) { + return toAsync(Actions.toFunc(action), scheduler); + } + /** + * Convert a synchronous function call into an asynchronous function + * call through an Observable sequence. + * + * @param func the function to convert + * @param scheduler the scheduler used to call the {@code func} + * + * @return a function which returns an observable sequence which + * executes the {@code func} and emits its returned value. + * + */ + public static FuncN> toAsync(final FuncN func, final Scheduler scheduler) { + return new FuncN>() { + @Override + public Observable call(final Object... args) { + final AsyncSubject subject = AsyncSubject.create(); + scheduler.schedule(new Action0() { + @Override + public void call() { + R result; + try { + result = func.call(args); + } catch (Throwable t) { + subject.onError(t); + return; + } + subject.onNext(result); + subject.onCompleted(); + } + }); + return subject; + } + }; + } + /** + * Convert a synchronous action call into an asynchronous function + * call through an Observable sequence. + *

+ * Alias for toAsync(ActionN) intended for dynamic languages. + * + * @param action the action to convert + * + * @return a function which returns an observable sequence which + * executes the {@code action} and emits {@code null}. + * + */ + public static FuncN> asyncAction(final ActionN action) { + return toAsync(action); + } + /** + * Convert a synchronous action call into an asynchronous function + * call through an Observable sequence. + *

+ * Alias for toAsync(ActionN, Scheduler) intended for dynamic languages. + * + * @param action the action to convert + * @param scheduler the scheduler used to execute the {@code action} + * + * @return a function which returns an observable sequence which + * executes the {@code action} and emits {@code null}. + * + */ + public static FuncN> asyncAction(final ActionN action, final Scheduler scheduler) { + return toAsync(action, scheduler); + } + /** + * Convert a synchronous function call into an asynchronous function + * call through an Observable sequence. + *

+ * Alias for toAsync(FuncN) intended for dynamic languages. + * + * @param func the function to convert + * + * @return a function which returns an observable sequence which + * executes the {@code func} and emits its returned value. + * + */ + public static FuncN> asyncFunc(final FuncN func) { + return toAsync(func); + } + /** + * Convert a synchronous function call into an asynchronous function + * call through an Observable sequence. + *

+ * Alias for toAsync(FuncN, Scheduler) intended for dynamic languages. + * + * @param func the function to convert + * @param scheduler the scheduler used to call the {@code func} + * + * @return a function which returns an observable sequence which + * executes the {@code func} and emits its returned value. + * + */ + public static FuncN> asyncFunc(final FuncN func, final Scheduler scheduler) { + return toAsync(func, scheduler); + } +} diff --git a/rxjava-core/src/test/java/rx/operators/OperationJoinsTest.java b/rxjava-core/src/test/java/rx/operators/OperationJoinsTest.java index 8a3b0c79c9..53b6dffbf6 100644 --- a/rxjava-core/src/test/java/rx/operators/OperationJoinsTest.java +++ b/rxjava-core/src/test/java/rx/operators/OperationJoinsTest.java @@ -1,382 +1,382 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package rx.operators; - -import org.junit.Before; -import org.junit.Test; -import org.mockito.InOrder; -import static org.mockito.Matchers.any; -import org.mockito.Mock; -import static org.mockito.Mockito.inOrder; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import org.mockito.MockitoAnnotations; -import rx.Observable; -import rx.Observer; -import rx.joins.Plan0; -import rx.subjects.PublishSubject; -import rx.util.functions.Func1; -import rx.util.functions.Func2; -import rx.util.functions.Func3; -import rx.util.functions.Functions; - -public class OperationJoinsTest { - @Mock - Observer observer; - - - Func2 add2 = new Func2() { - @Override - public Integer call(Integer t1, Integer t2) { - return t1 + t2; - } - }; - Func2 mul2 = new Func2() { - @Override - public Integer call(Integer t1, Integer t2) { - return t1 * t2; - } - }; - Func2 sub2 = new Func2() { - @Override - public Integer call(Integer t1, Integer t2) { - return t1 - t2; - } - }; - - Func3 add3 = new Func3() { - @Override - public Integer call(Integer t1, Integer t2, Integer t3) { - return t1 + t2 + t3; - } - }; - Func1 func1Throw = new Func1() { - @Override - public Integer call(Integer t1) { - throw new RuntimeException("Forced failure"); - } - }; - Func2 func2Throw = new Func2() { - @Override - public Integer call(Integer t1, Integer t2) { - throw new RuntimeException("Forced failure"); - } - }; - Func3 func3Throw = new Func3() { - @Override - public Integer call(Integer t1, Integer t2, Integer t3) { - throw new RuntimeException("Forced failure"); - } - }; - - @Before - public void before() { - MockitoAnnotations.initMocks(this); - } - - @Test(expected = NullPointerException.class) - public void and2ArgumentNull() { - Observable some = Observable.just(1); - some.and(null); - } - @Test(expected = NullPointerException.class) - public void and3argumentNull() { - Observable some = Observable.just(1); - some.and(some).and(null); - } - @Test - public void and2() { - Observable some = Observable.just(1); - - Observable m = Observable.when(some.and(some).then(add2)); - - m.subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(2); - verify(observer, times(1)).onCompleted(); - } - @Test - public void and2Error1() { - Observable error = Observable.error(new RuntimeException("Forced failure")); - - Observable some = Observable.just(1); - - Observable m = Observable.when(error.and(some).then(add2)); - - m.subscribe(observer); - - verify(observer, times(1)).onError(any(Throwable.class)); - verify(observer, never()).onNext(any()); - verify(observer, never()).onCompleted(); - } - @Test - public void and2Error2() { - Observable error = Observable.error(new RuntimeException("Forced failure")); - - Observable some = Observable.just(1); - - Observable m = Observable.when(some.and(error).then(add2)); - - m.subscribe(observer); - - verify(observer, times(1)).onError(any(Throwable.class)); - verify(observer, never()).onNext(any()); - verify(observer, never()).onCompleted(); - } - @Test - public void and3() { - Observable some = Observable.just(1); - - Observable m = Observable.when(some.and(some).and(some).then(add3)); - - m.subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(3); - verify(observer, times(1)).onCompleted(); - } - @Test - public void and3Error1() { - Observable error = Observable.error(new RuntimeException("Forced failure")); - - Observable some = Observable.just(1); - - Observable m = Observable.when(error.and(some).and(some).then(add3)); - - m.subscribe(observer); - - verify(observer, times(1)).onError(any(Throwable.class)); - verify(observer, never()).onNext(any()); - verify(observer, never()).onCompleted(); - } - @Test - public void and3Error2() { - Observable error = Observable.error(new RuntimeException("Forced failure")); - - Observable some = Observable.just(1); - - Observable m = Observable.when(some.and(error).and(some).then(add3)); - - m.subscribe(observer); - - verify(observer, times(1)).onError(any(Throwable.class)); - verify(observer, never()).onNext(any()); - verify(observer, never()).onCompleted(); - } - @Test - public void and3Error3() { - Observable error = Observable.error(new RuntimeException("Forced failure")); - - Observable some = Observable.just(1); - - Observable m = Observable.when(some.and(some).and(error).then(add3)); - - m.subscribe(observer); - - verify(observer, times(1)).onError(any(Throwable.class)); - verify(observer, never()).onNext(any()); - verify(observer, never()).onCompleted(); - } - @Test(expected = NullPointerException.class) - public void thenArgumentNull() { - Observable some = Observable.just(1); - - some.then(null); - } - @Test(expected = NullPointerException.class) - public void then2ArgumentNull() { - Observable some = Observable.just(1); - - some.and(some).then(null); - } - @Test(expected = NullPointerException.class) - public void then3ArgumentNull() { - Observable some = Observable.just(1); - - some.and(some).and(some).then(null); - } - @Test - public void then1() { - Observable some = Observable.just(1); - - Observable m = Observable.when(some.then(Functions.identity())); - m.subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(1); - verify(observer, times(1)).onCompleted(); - } - @Test - public void then1Error() { - Observable some = Observable.error(new RuntimeException("Forced failure")); - - Observable m = Observable.when(some.then(Functions.identity())); - m.subscribe(observer); - - verify(observer, times(1)).onError(any(Throwable.class)); - verify(observer, never()).onNext(any()); - verify(observer, never()).onCompleted(); - } - @Test - public void then1Throws() { - Observable some = Observable.just(1); - - Observable m = Observable.when(some.then(func1Throw)); - m.subscribe(observer); - - verify(observer, times(1)).onError(any(Throwable.class)); - verify(observer, never()).onNext(any()); - verify(observer, never()).onCompleted(); - } - @Test - public void then2Throws() { - Observable some = Observable.just(1); - - Observable m = Observable.when(some.and(some).then(func2Throw)); - m.subscribe(observer); - - verify(observer, times(1)).onError(any(Throwable.class)); - verify(observer, never()).onNext(any()); - verify(observer, never()).onCompleted(); - } - @Test - public void then3Throws() { - Observable some = Observable.just(1); - - Observable m = Observable.when(some.and(some).and(some).then(func3Throw)); - m.subscribe(observer); - - verify(observer, times(1)).onError(any(Throwable.class)); - verify(observer, never()).onNext(any()); - verify(observer, never()).onCompleted(); - } - @Test(expected = NullPointerException.class) - public void whenArgumentNull1() { - Observable.when((Plan0[])null); - } - @Test(expected = NullPointerException.class) - public void whenArgumentNull2() { - Observable.when((Iterable>)null); - } - @Test - public void whenMultipleSymmetric() { - Observable source1 = Observable.from(1, 2, 3); - Observable source2 = Observable.from(4, 5, 6); - - Observable m = Observable.when(source1.and(source2).then(add2)); - m.subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(1 + 4); - verify(observer, times(1)).onNext(2 + 5); - verify(observer, times(1)).onNext(3 + 6); - verify(observer, times(1)).onCompleted(); - } - - @Test - public void whenMultipleAsymSymmetric() { - Observable source1 = Observable.from(1, 2, 3); - Observable source2 = Observable.from(4, 5); - - Observable m = Observable.when(source1.and(source2).then(add2)); - m.subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(1 + 4); - verify(observer, times(1)).onNext(2 + 5); - verify(observer, times(1)).onCompleted(); - } - @Test - public void whenEmptyEmpty() { - Observable source1 = Observable.empty(); - Observable source2 = Observable.empty(); - - Observable m = Observable.when(source1.and(source2).then(add2)); - m.subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, never()).onNext(any()); - verify(observer, times(1)).onCompleted(); - } - - @Test - public void whenNeverNever() { - Observable source1 = Observable.never(); - Observable source2 = Observable.never(); - - Observable m = Observable.when(source1.and(source2).then(add2)); - m.subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, never()).onNext(any()); - verify(observer, never()).onCompleted(); - } - @Test - public void whenThrowNonEmpty() { - Observable source1 = Observable.empty(); - Observable source2 = Observable.error(new RuntimeException("Forced failure")); - - Observable m = Observable.when(source1.and(source2).then(add2)); - m.subscribe(observer); - - verify(observer, times(1)).onError(any(Throwable.class)); - verify(observer, never()).onNext(any()); - verify(observer, never()).onCompleted(); - } - @Test - public void whenComplicated() { - PublishSubject xs = PublishSubject.create(); - PublishSubject ys = PublishSubject.create(); - PublishSubject zs = PublishSubject.create(); - - Observable m = Observable.when( - xs.and(ys).then(add2), - xs.and(zs).then(mul2), - ys.and(zs).then(sub2) - ); - m.subscribe(observer); - - xs.onNext(1); // t == 210 - - xs.onNext(2); // t == 220 - zs.onNext(7); // t == 220 - - xs.onNext(3); // t == 230 - zs.onNext(8); // t == 230 - - ys.onNext(4); // t == 240 - zs.onNext(9); // t == 240 - xs.onCompleted(); // t == 240 - - ys.onNext(5); // t == 250 - - ys.onNext(6); // t == 260 - - ys.onCompleted(); // t == 270 - - zs.onCompleted(); // t == 300 - - InOrder inOrder = inOrder(observer); - inOrder.verify(observer, times(1)).onNext(1 * 7); - inOrder.verify(observer, times(1)).onNext(2 * 8); - inOrder.verify(observer, times(1)).onNext(3 + 4); - inOrder.verify(observer, times(1)).onNext(5 - 9); - inOrder.verify(observer, times(1)).onCompleted(); - verify(observer, never()).onError(any(Throwable.class)); - } -} +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ +package rx.operators; + +import org.junit.Before; +import org.junit.Test; +import org.mockito.InOrder; +import static org.mockito.Matchers.any; +import org.mockito.Mock; +import static org.mockito.Mockito.inOrder; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import org.mockito.MockitoAnnotations; +import rx.Observable; +import rx.Observer; +import rx.joins.Plan0; +import rx.subjects.PublishSubject; +import rx.util.functions.Func1; +import rx.util.functions.Func2; +import rx.util.functions.Func3; +import rx.util.functions.Functions; + +public class OperationJoinsTest { + @Mock + Observer observer; + + + Func2 add2 = new Func2() { + @Override + public Integer call(Integer t1, Integer t2) { + return t1 + t2; + } + }; + Func2 mul2 = new Func2() { + @Override + public Integer call(Integer t1, Integer t2) { + return t1 * t2; + } + }; + Func2 sub2 = new Func2() { + @Override + public Integer call(Integer t1, Integer t2) { + return t1 - t2; + } + }; + + Func3 add3 = new Func3() { + @Override + public Integer call(Integer t1, Integer t2, Integer t3) { + return t1 + t2 + t3; + } + }; + Func1 func1Throw = new Func1() { + @Override + public Integer call(Integer t1) { + throw new RuntimeException("Forced failure"); + } + }; + Func2 func2Throw = new Func2() { + @Override + public Integer call(Integer t1, Integer t2) { + throw new RuntimeException("Forced failure"); + } + }; + Func3 func3Throw = new Func3() { + @Override + public Integer call(Integer t1, Integer t2, Integer t3) { + throw new RuntimeException("Forced failure"); + } + }; + + @Before + public void before() { + MockitoAnnotations.initMocks(this); + } + + @Test(expected = NullPointerException.class) + public void and2ArgumentNull() { + Observable some = Observable.just(1); + some.and(null); + } + @Test(expected = NullPointerException.class) + public void and3argumentNull() { + Observable some = Observable.just(1); + some.and(some).and(null); + } + @Test + public void and2() { + Observable some = Observable.just(1); + + Observable m = Observable.when(some.and(some).then(add2)); + + m.subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(2); + verify(observer, times(1)).onCompleted(); + } + @Test + public void and2Error1() { + Observable error = Observable.error(new RuntimeException("Forced failure")); + + Observable some = Observable.just(1); + + Observable m = Observable.when(error.and(some).then(add2)); + + m.subscribe(observer); + + verify(observer, times(1)).onError(any(Throwable.class)); + verify(observer, never()).onNext(any()); + verify(observer, never()).onCompleted(); + } + @Test + public void and2Error2() { + Observable error = Observable.error(new RuntimeException("Forced failure")); + + Observable some = Observable.just(1); + + Observable m = Observable.when(some.and(error).then(add2)); + + m.subscribe(observer); + + verify(observer, times(1)).onError(any(Throwable.class)); + verify(observer, never()).onNext(any()); + verify(observer, never()).onCompleted(); + } + @Test + public void and3() { + Observable some = Observable.just(1); + + Observable m = Observable.when(some.and(some).and(some).then(add3)); + + m.subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(3); + verify(observer, times(1)).onCompleted(); + } + @Test + public void and3Error1() { + Observable error = Observable.error(new RuntimeException("Forced failure")); + + Observable some = Observable.just(1); + + Observable m = Observable.when(error.and(some).and(some).then(add3)); + + m.subscribe(observer); + + verify(observer, times(1)).onError(any(Throwable.class)); + verify(observer, never()).onNext(any()); + verify(observer, never()).onCompleted(); + } + @Test + public void and3Error2() { + Observable error = Observable.error(new RuntimeException("Forced failure")); + + Observable some = Observable.just(1); + + Observable m = Observable.when(some.and(error).and(some).then(add3)); + + m.subscribe(observer); + + verify(observer, times(1)).onError(any(Throwable.class)); + verify(observer, never()).onNext(any()); + verify(observer, never()).onCompleted(); + } + @Test + public void and3Error3() { + Observable error = Observable.error(new RuntimeException("Forced failure")); + + Observable some = Observable.just(1); + + Observable m = Observable.when(some.and(some).and(error).then(add3)); + + m.subscribe(observer); + + verify(observer, times(1)).onError(any(Throwable.class)); + verify(observer, never()).onNext(any()); + verify(observer, never()).onCompleted(); + } + @Test(expected = NullPointerException.class) + public void thenArgumentNull() { + Observable some = Observable.just(1); + + some.then(null); + } + @Test(expected = NullPointerException.class) + public void then2ArgumentNull() { + Observable some = Observable.just(1); + + some.and(some).then(null); + } + @Test(expected = NullPointerException.class) + public void then3ArgumentNull() { + Observable some = Observable.just(1); + + some.and(some).and(some).then(null); + } + @Test + public void then1() { + Observable some = Observable.just(1); + + Observable m = Observable.when(some.then(Functions.identity())); + m.subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(1); + verify(observer, times(1)).onCompleted(); + } + @Test + public void then1Error() { + Observable some = Observable.error(new RuntimeException("Forced failure")); + + Observable m = Observable.when(some.then(Functions.identity())); + m.subscribe(observer); + + verify(observer, times(1)).onError(any(Throwable.class)); + verify(observer, never()).onNext(any()); + verify(observer, never()).onCompleted(); + } + @Test + public void then1Throws() { + Observable some = Observable.just(1); + + Observable m = Observable.when(some.then(func1Throw)); + m.subscribe(observer); + + verify(observer, times(1)).onError(any(Throwable.class)); + verify(observer, never()).onNext(any()); + verify(observer, never()).onCompleted(); + } + @Test + public void then2Throws() { + Observable some = Observable.just(1); + + Observable m = Observable.when(some.and(some).then(func2Throw)); + m.subscribe(observer); + + verify(observer, times(1)).onError(any(Throwable.class)); + verify(observer, never()).onNext(any()); + verify(observer, never()).onCompleted(); + } + @Test + public void then3Throws() { + Observable some = Observable.just(1); + + Observable m = Observable.when(some.and(some).and(some).then(func3Throw)); + m.subscribe(observer); + + verify(observer, times(1)).onError(any(Throwable.class)); + verify(observer, never()).onNext(any()); + verify(observer, never()).onCompleted(); + } + @Test(expected = NullPointerException.class) + public void whenArgumentNull1() { + Observable.when((Plan0[])null); + } + @Test(expected = NullPointerException.class) + public void whenArgumentNull2() { + Observable.when((Iterable>)null); + } + @Test + public void whenMultipleSymmetric() { + Observable source1 = Observable.from(1, 2, 3); + Observable source2 = Observable.from(4, 5, 6); + + Observable m = Observable.when(source1.and(source2).then(add2)); + m.subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(1 + 4); + verify(observer, times(1)).onNext(2 + 5); + verify(observer, times(1)).onNext(3 + 6); + verify(observer, times(1)).onCompleted(); + } + + @Test + public void whenMultipleAsymSymmetric() { + Observable source1 = Observable.from(1, 2, 3); + Observable source2 = Observable.from(4, 5); + + Observable m = Observable.when(source1.and(source2).then(add2)); + m.subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(1 + 4); + verify(observer, times(1)).onNext(2 + 5); + verify(observer, times(1)).onCompleted(); + } + @Test + public void whenEmptyEmpty() { + Observable source1 = Observable.empty(); + Observable source2 = Observable.empty(); + + Observable m = Observable.when(source1.and(source2).then(add2)); + m.subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, never()).onNext(any()); + verify(observer, times(1)).onCompleted(); + } + + @Test + public void whenNeverNever() { + Observable source1 = Observable.never(); + Observable source2 = Observable.never(); + + Observable m = Observable.when(source1.and(source2).then(add2)); + m.subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, never()).onNext(any()); + verify(observer, never()).onCompleted(); + } + @Test + public void whenThrowNonEmpty() { + Observable source1 = Observable.empty(); + Observable source2 = Observable.error(new RuntimeException("Forced failure")); + + Observable m = Observable.when(source1.and(source2).then(add2)); + m.subscribe(observer); + + verify(observer, times(1)).onError(any(Throwable.class)); + verify(observer, never()).onNext(any()); + verify(observer, never()).onCompleted(); + } + @Test + public void whenComplicated() { + PublishSubject xs = PublishSubject.create(); + PublishSubject ys = PublishSubject.create(); + PublishSubject zs = PublishSubject.create(); + + Observable m = Observable.when( + xs.and(ys).then(add2), + xs.and(zs).then(mul2), + ys.and(zs).then(sub2) + ); + m.subscribe(observer); + + xs.onNext(1); // t == 210 + + xs.onNext(2); // t == 220 + zs.onNext(7); // t == 220 + + xs.onNext(3); // t == 230 + zs.onNext(8); // t == 230 + + ys.onNext(4); // t == 240 + zs.onNext(9); // t == 240 + xs.onCompleted(); // t == 240 + + ys.onNext(5); // t == 250 + + ys.onNext(6); // t == 260 + + ys.onCompleted(); // t == 270 + + zs.onCompleted(); // t == 300 + + InOrder inOrder = inOrder(observer); + inOrder.verify(observer, times(1)).onNext(1 * 7); + inOrder.verify(observer, times(1)).onNext(2 * 8); + inOrder.verify(observer, times(1)).onNext(3 + 4); + inOrder.verify(observer, times(1)).onNext(5 - 9); + inOrder.verify(observer, times(1)).onCompleted(); + verify(observer, never()).onError(any(Throwable.class)); + } +} diff --git a/rxjava-core/src/test/java/rx/operators/OperationToMapTest.java b/rxjava-core/src/test/java/rx/operators/OperationToMapTest.java index ab1a9fb9de..f35d555a34 100644 --- a/rxjava-core/src/test/java/rx/operators/OperationToMapTest.java +++ b/rxjava-core/src/test/java/rx/operators/OperationToMapTest.java @@ -1,215 +1,215 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package rx.operators; - -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.Map; -import org.junit.Test; -import org.junit.Before; -import org.mockito.*; -import static org.mockito.Mockito.*; -import rx.Observable; -import rx.Observer; -import rx.util.functions.Func0; -import rx.util.functions.Func1; -import rx.util.functions.Functions; - -public class OperationToMapTest { - @Mock - Observer objectObserver; - - @Before - public void before() { - MockitoAnnotations.initMocks(this); - } - Func1 lengthFunc = new Func1() { - @Override - public Integer call(String t1) { - return t1.length(); - } - }; - Func1 duplicate = new Func1() { - @Override - public String call(String t1) { - return t1 + t1; - } - }; - @Test - public void testToMap() { - Observable source = Observable.from("a", "bb", "ccc", "dddd"); - - - Observable> mapped = Observable.create(OperationToMap.toMap(source, lengthFunc)); - - Map expected = new HashMap(); - expected.put(1, "a"); - expected.put(2, "bb"); - expected.put(3, "ccc"); - expected.put(4, "dddd"); - - mapped.subscribe(objectObserver); - - verify(objectObserver, never()).onError(any(Throwable.class)); - verify(objectObserver, times(1)).onNext(expected); - verify(objectObserver, times(1)).onCompleted(); - } - - @Test - public void testToMapWithValueSelector() { - Observable source = Observable.from("a", "bb", "ccc", "dddd"); - - Observable> mapped = Observable.create(OperationToMap.toMap(source, lengthFunc, duplicate)); - - Map expected = new HashMap(); - expected.put(1, "aa"); - expected.put(2, "bbbb"); - expected.put(3, "cccccc"); - expected.put(4, "dddddddd"); - - mapped.subscribe(objectObserver); - - verify(objectObserver, never()).onError(any(Throwable.class)); - verify(objectObserver, times(1)).onNext(expected); - verify(objectObserver, times(1)).onCompleted(); - } - - @Test - public void testToMapWithError() { - Observable source = Observable.from("a", "bb", "ccc", "dddd"); - - Func1 lengthFuncErr = new Func1() { - @Override - public Integer call(String t1) { - if ("bb".equals(t1)) { - throw new RuntimeException("Forced Failure"); - } - return t1.length(); - } - }; - Observable> mapped = Observable.create(OperationToMap.toMap(source, lengthFuncErr)); - - Map expected = new HashMap(); - expected.put(1, "a"); - expected.put(2, "bb"); - expected.put(3, "ccc"); - expected.put(4, "dddd"); - - mapped.subscribe(objectObserver); - - verify(objectObserver, never()).onNext(expected); - verify(objectObserver, never()).onCompleted(); - verify(objectObserver, times(1)).onError(any(Throwable.class)); - - } - - @Test - public void testToMapWithErrorInValueSelector() { - Observable source = Observable.from("a", "bb", "ccc", "dddd"); - - Func1 duplicateErr = new Func1() { - @Override - public String call(String t1) { - if ("bb".equals(t1)) { - throw new RuntimeException("Forced failure"); - } - return t1 + t1; - } - }; - - Observable> mapped = Observable.create(OperationToMap.toMap(source, lengthFunc, duplicateErr)); - - Map expected = new HashMap(); - expected.put(1, "aa"); - expected.put(2, "bbbb"); - expected.put(3, "cccccc"); - expected.put(4, "dddddddd"); - - mapped.subscribe(objectObserver); - - verify(objectObserver, never()).onNext(expected); - verify(objectObserver, never()).onCompleted(); - verify(objectObserver, times(1)).onError(any(Throwable.class)); - - } - - @Test - public void testToMapWithFactory() { - Observable source = Observable.from("a", "bb", "ccc", "dddd"); - - Func0> mapFactory = new Func0>() { - @Override - public Map call() { - return new LinkedHashMap() { - @Override - protected boolean removeEldestEntry(Map.Entry eldest) { - return size() > 3; - } - }; - } - }; - - Func1 lengthFunc = new Func1() { - @Override - public Integer call(String t1) { - return t1.length(); - } - }; - Observable> mapped = Observable.create(OperationToMap.toMap(source, lengthFunc, Functions.identity(), mapFactory)); - - Map expected = new LinkedHashMap(); - expected.put(2, "bb"); - expected.put(3, "ccc"); - expected.put(4, "dddd"); - - mapped.subscribe(objectObserver); - - verify(objectObserver, never()).onError(any(Throwable.class)); - verify(objectObserver, times(1)).onNext(expected); - verify(objectObserver, times(1)).onCompleted(); - } - @Test - public void testToMapWithErrorThrowingFactory() { - Observable source = Observable.from("a", "bb", "ccc", "dddd"); - - Func0> mapFactory = new Func0>() { - @Override - public Map call() { - throw new RuntimeException("Forced failure"); - } - }; - - Func1 lengthFunc = new Func1() { - @Override - public Integer call(String t1) { - return t1.length(); - } - }; - Observable> mapped = Observable.create(OperationToMap.toMap(source, lengthFunc, Functions.identity(), mapFactory)); - - Map expected = new LinkedHashMap(); - expected.put(2, "bb"); - expected.put(3, "ccc"); - expected.put(4, "dddd"); - - mapped.subscribe(objectObserver); - - verify(objectObserver, never()).onNext(expected); - verify(objectObserver, never()).onCompleted(); - verify(objectObserver, times(1)).onError(any(Throwable.class)); - } - -} +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package rx.operators; + +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.Map; +import org.junit.Test; +import org.junit.Before; +import org.mockito.*; +import static org.mockito.Mockito.*; +import rx.Observable; +import rx.Observer; +import rx.util.functions.Func0; +import rx.util.functions.Func1; +import rx.util.functions.Functions; + +public class OperationToMapTest { + @Mock + Observer objectObserver; + + @Before + public void before() { + MockitoAnnotations.initMocks(this); + } + Func1 lengthFunc = new Func1() { + @Override + public Integer call(String t1) { + return t1.length(); + } + }; + Func1 duplicate = new Func1() { + @Override + public String call(String t1) { + return t1 + t1; + } + }; + @Test + public void testToMap() { + Observable source = Observable.from("a", "bb", "ccc", "dddd"); + + + Observable> mapped = Observable.create(OperationToMap.toMap(source, lengthFunc)); + + Map expected = new HashMap(); + expected.put(1, "a"); + expected.put(2, "bb"); + expected.put(3, "ccc"); + expected.put(4, "dddd"); + + mapped.subscribe(objectObserver); + + verify(objectObserver, never()).onError(any(Throwable.class)); + verify(objectObserver, times(1)).onNext(expected); + verify(objectObserver, times(1)).onCompleted(); + } + + @Test + public void testToMapWithValueSelector() { + Observable source = Observable.from("a", "bb", "ccc", "dddd"); + + Observable> mapped = Observable.create(OperationToMap.toMap(source, lengthFunc, duplicate)); + + Map expected = new HashMap(); + expected.put(1, "aa"); + expected.put(2, "bbbb"); + expected.put(3, "cccccc"); + expected.put(4, "dddddddd"); + + mapped.subscribe(objectObserver); + + verify(objectObserver, never()).onError(any(Throwable.class)); + verify(objectObserver, times(1)).onNext(expected); + verify(objectObserver, times(1)).onCompleted(); + } + + @Test + public void testToMapWithError() { + Observable source = Observable.from("a", "bb", "ccc", "dddd"); + + Func1 lengthFuncErr = new Func1() { + @Override + public Integer call(String t1) { + if ("bb".equals(t1)) { + throw new RuntimeException("Forced Failure"); + } + return t1.length(); + } + }; + Observable> mapped = Observable.create(OperationToMap.toMap(source, lengthFuncErr)); + + Map expected = new HashMap(); + expected.put(1, "a"); + expected.put(2, "bb"); + expected.put(3, "ccc"); + expected.put(4, "dddd"); + + mapped.subscribe(objectObserver); + + verify(objectObserver, never()).onNext(expected); + verify(objectObserver, never()).onCompleted(); + verify(objectObserver, times(1)).onError(any(Throwable.class)); + + } + + @Test + public void testToMapWithErrorInValueSelector() { + Observable source = Observable.from("a", "bb", "ccc", "dddd"); + + Func1 duplicateErr = new Func1() { + @Override + public String call(String t1) { + if ("bb".equals(t1)) { + throw new RuntimeException("Forced failure"); + } + return t1 + t1; + } + }; + + Observable> mapped = Observable.create(OperationToMap.toMap(source, lengthFunc, duplicateErr)); + + Map expected = new HashMap(); + expected.put(1, "aa"); + expected.put(2, "bbbb"); + expected.put(3, "cccccc"); + expected.put(4, "dddddddd"); + + mapped.subscribe(objectObserver); + + verify(objectObserver, never()).onNext(expected); + verify(objectObserver, never()).onCompleted(); + verify(objectObserver, times(1)).onError(any(Throwable.class)); + + } + + @Test + public void testToMapWithFactory() { + Observable source = Observable.from("a", "bb", "ccc", "dddd"); + + Func0> mapFactory = new Func0>() { + @Override + public Map call() { + return new LinkedHashMap() { + @Override + protected boolean removeEldestEntry(Map.Entry eldest) { + return size() > 3; + } + }; + } + }; + + Func1 lengthFunc = new Func1() { + @Override + public Integer call(String t1) { + return t1.length(); + } + }; + Observable> mapped = Observable.create(OperationToMap.toMap(source, lengthFunc, Functions.identity(), mapFactory)); + + Map expected = new LinkedHashMap(); + expected.put(2, "bb"); + expected.put(3, "ccc"); + expected.put(4, "dddd"); + + mapped.subscribe(objectObserver); + + verify(objectObserver, never()).onError(any(Throwable.class)); + verify(objectObserver, times(1)).onNext(expected); + verify(objectObserver, times(1)).onCompleted(); + } + @Test + public void testToMapWithErrorThrowingFactory() { + Observable source = Observable.from("a", "bb", "ccc", "dddd"); + + Func0> mapFactory = new Func0>() { + @Override + public Map call() { + throw new RuntimeException("Forced failure"); + } + }; + + Func1 lengthFunc = new Func1() { + @Override + public Integer call(String t1) { + return t1.length(); + } + }; + Observable> mapped = Observable.create(OperationToMap.toMap(source, lengthFunc, Functions.identity(), mapFactory)); + + Map expected = new LinkedHashMap(); + expected.put(2, "bb"); + expected.put(3, "ccc"); + expected.put(4, "dddd"); + + mapped.subscribe(objectObserver); + + verify(objectObserver, never()).onNext(expected); + verify(objectObserver, never()).onCompleted(); + verify(objectObserver, times(1)).onError(any(Throwable.class)); + } + +} diff --git a/rxjava-core/src/test/java/rx/operators/OperationToMultimapTest.java b/rxjava-core/src/test/java/rx/operators/OperationToMultimapTest.java index f715e69c61..b254d93d2e 100644 --- a/rxjava-core/src/test/java/rx/operators/OperationToMultimapTest.java +++ b/rxjava-core/src/test/java/rx/operators/OperationToMultimapTest.java @@ -1,250 +1,250 @@ -/* - * To change this license header, choose License Headers in Project Properties. - * To change this template file, choose Tools | Templates - * and open the template in the editor. - */ - -package rx.operators; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedHashMap; -import java.util.Map; -import org.junit.Test; -import org.junit.Before; -import org.mockito.*; -import static org.mockito.Mockito.*; -import rx.Observable; -import rx.Observer; -import rx.operators.OperationToMultimap.DefaultMultimapCollectionFactory; -import rx.operators.OperationToMultimap.DefaultToMultimapFactory; -import rx.util.functions.Func0; -import rx.util.functions.Func1; -import rx.util.functions.Functions; -public class OperationToMultimapTest { - @Mock - Observer objectObserver; - - @Before - public void before() { - MockitoAnnotations.initMocks(this); - } - Func1 lengthFunc = new Func1() { - @Override - public Integer call(String t1) { - return t1.length(); - } - }; - Func1 duplicate = new Func1() { - @Override - public String call(String t1) { - return t1 + t1; - } - }; - @Test - public void testToMultimap() { - Observable source = Observable.from("a", "b", "cc", "dd"); - - - Observable>> mapped = Observable.create(OperationToMultimap.toMultimap(source, lengthFunc)); - - Map> expected = new HashMap>(); - expected.put(1, Arrays.asList("a", "b")); - expected.put(2, Arrays.asList("cc", "dd")); - - mapped.subscribe(objectObserver); - - verify(objectObserver, never()).onError(any(Throwable.class)); - verify(objectObserver, times(1)).onNext(expected); - verify(objectObserver, times(1)).onCompleted(); - } - @Test - public void testToMultimapWithValueSelector() { - Observable source = Observable.from("a", "b", "cc", "dd"); - - - Observable>> mapped = Observable.create(OperationToMultimap.toMultimap(source, lengthFunc, duplicate)); - - Map> expected = new HashMap>(); - expected.put(1, Arrays.asList("aa", "bb")); - expected.put(2, Arrays.asList("cccc", "dddd")); - - mapped.subscribe(objectObserver); - - verify(objectObserver, never()).onError(any(Throwable.class)); - verify(objectObserver, times(1)).onNext(expected); - verify(objectObserver, times(1)).onCompleted(); - } - @Test - public void testToMultimapWithMapFactory() { - Observable source = Observable.from("a", "b", "cc", "dd", "eee", "fff"); - - Func0>> mapFactory = new Func0>>() { - @Override - public Map> call() { - return new LinkedHashMap>() { - @Override - protected boolean removeEldestEntry(Map.Entry> eldest) { - return size() > 2; - } - }; - } - }; - - Observable>> mapped = Observable.create( - OperationToMultimap.toMultimap(source, - lengthFunc, Functions.identity(), - mapFactory, new DefaultMultimapCollectionFactory())); - - Map> expected = new HashMap>(); - expected.put(2, Arrays.asList("cc", "dd")); - expected.put(3, Arrays.asList("eee", "fff")); - - mapped.subscribe(objectObserver); - - verify(objectObserver, never()).onError(any(Throwable.class)); - verify(objectObserver, times(1)).onNext(expected); - verify(objectObserver, times(1)).onCompleted(); - } - @Test - public void testToMultimapWithCollectionFactory() { - Observable source = Observable.from("cc", "dd", "eee", "eee"); - - Func1> collectionFactory = new Func1>() { - - @Override - public Collection call(Integer t1) { - if (t1 == 2) { - return new ArrayList(); - } else { - return new HashSet(); - } - } - }; - - Observable>> mapped = Observable.create( - OperationToMultimap.toMultimap( - source, lengthFunc, Functions.identity(), - new DefaultToMultimapFactory(), collectionFactory)); - - Map> expected = new HashMap>(); - expected.put(2, Arrays.asList("cc", "dd")); - expected.put(3, new HashSet(Arrays.asList("eee"))); - - mapped.subscribe(objectObserver); - - verify(objectObserver, never()).onError(any(Throwable.class)); - verify(objectObserver, times(1)).onNext(expected); - verify(objectObserver, times(1)).onCompleted(); - } - @Test - public void testToMultimapWithError() { - Observable source = Observable.from("a", "b", "cc", "dd"); - - Func1 lengthFuncErr = new Func1() { - @Override - public Integer call(String t1) { - if ("b".equals(t1)) { - throw new RuntimeException("Forced Failure"); - } - return t1.length(); - } - }; - - Observable>> mapped = Observable.create(OperationToMultimap.toMultimap(source, lengthFuncErr)); - - Map> expected = new HashMap>(); - expected.put(1, Arrays.asList("a", "b")); - expected.put(2, Arrays.asList("cc", "dd")); - - mapped.subscribe(objectObserver); - - verify(objectObserver, times(1)).onError(any(Throwable.class)); - verify(objectObserver, never()).onNext(expected); - verify(objectObserver, never()).onCompleted(); - } - @Test - public void testToMultimapWithErrorInValueSelector() { - Observable source = Observable.from("a", "b", "cc", "dd"); - - Func1 duplicateErr = new Func1() { - @Override - public String call(String t1) { - if ("b".equals(t1)) { - throw new RuntimeException("Forced failure"); - } - return t1 + t1; - } - }; - - Observable>> mapped = Observable.create(OperationToMultimap.toMultimap(source, lengthFunc, duplicateErr)); - - Map> expected = new HashMap>(); - expected.put(1, Arrays.asList("aa", "bb")); - expected.put(2, Arrays.asList("cccc", "dddd")); - - mapped.subscribe(objectObserver); - - verify(objectObserver, times(1)).onError(any(Throwable.class)); - verify(objectObserver, never()).onNext(expected); - verify(objectObserver, never()).onCompleted(); - } - - @Test - public void testToMultimapWithMapThrowingFactory() { - Observable source = Observable.from("a", "b", "cc", "dd", "eee", "fff"); - - Func0>> mapFactory = new Func0>>() { - @Override - public Map> call() { - throw new RuntimeException("Forced failure"); - } - }; - - Observable>> mapped = Observable.create( - OperationToMultimap.toMultimap(source, lengthFunc, Functions.identity(), mapFactory)); - - Map> expected = new HashMap>(); - expected.put(2, Arrays.asList("cc", "dd")); - expected.put(3, Arrays.asList("eee", "fff")); - - mapped.subscribe(objectObserver); - - verify(objectObserver, times(1)).onError(any(Throwable.class)); - verify(objectObserver, never()).onNext(expected); - verify(objectObserver, never()).onCompleted(); - } - @Test - public void testToMultimapWithThrowingCollectionFactory() { - Observable source = Observable.from("cc", "cc", "eee", "eee"); - - Func1> collectionFactory = new Func1>() { - - @Override - public Collection call(Integer t1) { - if (t1 == 2) { - throw new RuntimeException("Forced failure"); - } else { - return new HashSet(); - } - } - }; - - Observable>> mapped = Observable.create( - OperationToMultimap.toMultimap( - source, lengthFunc, Functions.identity(), new DefaultToMultimapFactory(), collectionFactory)); - - Map> expected = new HashMap>(); - expected.put(2, Arrays.asList("cc", "dd")); - expected.put(3, Collections.singleton("eee")); - - mapped.subscribe(objectObserver); - - verify(objectObserver, times(1)).onError(any(Throwable.class)); - verify(objectObserver, never()).onNext(expected); - verify(objectObserver, never()).onCompleted(); - } -} +/* + * To change this license header, choose License Headers in Project Properties. + * To change this template file, choose Tools | Templates + * and open the template in the editor. + */ + +package rx.operators; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.Map; +import org.junit.Test; +import org.junit.Before; +import org.mockito.*; +import static org.mockito.Mockito.*; +import rx.Observable; +import rx.Observer; +import rx.operators.OperationToMultimap.DefaultMultimapCollectionFactory; +import rx.operators.OperationToMultimap.DefaultToMultimapFactory; +import rx.util.functions.Func0; +import rx.util.functions.Func1; +import rx.util.functions.Functions; +public class OperationToMultimapTest { + @Mock + Observer objectObserver; + + @Before + public void before() { + MockitoAnnotations.initMocks(this); + } + Func1 lengthFunc = new Func1() { + @Override + public Integer call(String t1) { + return t1.length(); + } + }; + Func1 duplicate = new Func1() { + @Override + public String call(String t1) { + return t1 + t1; + } + }; + @Test + public void testToMultimap() { + Observable source = Observable.from("a", "b", "cc", "dd"); + + + Observable>> mapped = Observable.create(OperationToMultimap.toMultimap(source, lengthFunc)); + + Map> expected = new HashMap>(); + expected.put(1, Arrays.asList("a", "b")); + expected.put(2, Arrays.asList("cc", "dd")); + + mapped.subscribe(objectObserver); + + verify(objectObserver, never()).onError(any(Throwable.class)); + verify(objectObserver, times(1)).onNext(expected); + verify(objectObserver, times(1)).onCompleted(); + } + @Test + public void testToMultimapWithValueSelector() { + Observable source = Observable.from("a", "b", "cc", "dd"); + + + Observable>> mapped = Observable.create(OperationToMultimap.toMultimap(source, lengthFunc, duplicate)); + + Map> expected = new HashMap>(); + expected.put(1, Arrays.asList("aa", "bb")); + expected.put(2, Arrays.asList("cccc", "dddd")); + + mapped.subscribe(objectObserver); + + verify(objectObserver, never()).onError(any(Throwable.class)); + verify(objectObserver, times(1)).onNext(expected); + verify(objectObserver, times(1)).onCompleted(); + } + @Test + public void testToMultimapWithMapFactory() { + Observable source = Observable.from("a", "b", "cc", "dd", "eee", "fff"); + + Func0>> mapFactory = new Func0>>() { + @Override + public Map> call() { + return new LinkedHashMap>() { + @Override + protected boolean removeEldestEntry(Map.Entry> eldest) { + return size() > 2; + } + }; + } + }; + + Observable>> mapped = Observable.create( + OperationToMultimap.toMultimap(source, + lengthFunc, Functions.identity(), + mapFactory, new DefaultMultimapCollectionFactory())); + + Map> expected = new HashMap>(); + expected.put(2, Arrays.asList("cc", "dd")); + expected.put(3, Arrays.asList("eee", "fff")); + + mapped.subscribe(objectObserver); + + verify(objectObserver, never()).onError(any(Throwable.class)); + verify(objectObserver, times(1)).onNext(expected); + verify(objectObserver, times(1)).onCompleted(); + } + @Test + public void testToMultimapWithCollectionFactory() { + Observable source = Observable.from("cc", "dd", "eee", "eee"); + + Func1> collectionFactory = new Func1>() { + + @Override + public Collection call(Integer t1) { + if (t1 == 2) { + return new ArrayList(); + } else { + return new HashSet(); + } + } + }; + + Observable>> mapped = Observable.create( + OperationToMultimap.toMultimap( + source, lengthFunc, Functions.identity(), + new DefaultToMultimapFactory(), collectionFactory)); + + Map> expected = new HashMap>(); + expected.put(2, Arrays.asList("cc", "dd")); + expected.put(3, new HashSet(Arrays.asList("eee"))); + + mapped.subscribe(objectObserver); + + verify(objectObserver, never()).onError(any(Throwable.class)); + verify(objectObserver, times(1)).onNext(expected); + verify(objectObserver, times(1)).onCompleted(); + } + @Test + public void testToMultimapWithError() { + Observable source = Observable.from("a", "b", "cc", "dd"); + + Func1 lengthFuncErr = new Func1() { + @Override + public Integer call(String t1) { + if ("b".equals(t1)) { + throw new RuntimeException("Forced Failure"); + } + return t1.length(); + } + }; + + Observable>> mapped = Observable.create(OperationToMultimap.toMultimap(source, lengthFuncErr)); + + Map> expected = new HashMap>(); + expected.put(1, Arrays.asList("a", "b")); + expected.put(2, Arrays.asList("cc", "dd")); + + mapped.subscribe(objectObserver); + + verify(objectObserver, times(1)).onError(any(Throwable.class)); + verify(objectObserver, never()).onNext(expected); + verify(objectObserver, never()).onCompleted(); + } + @Test + public void testToMultimapWithErrorInValueSelector() { + Observable source = Observable.from("a", "b", "cc", "dd"); + + Func1 duplicateErr = new Func1() { + @Override + public String call(String t1) { + if ("b".equals(t1)) { + throw new RuntimeException("Forced failure"); + } + return t1 + t1; + } + }; + + Observable>> mapped = Observable.create(OperationToMultimap.toMultimap(source, lengthFunc, duplicateErr)); + + Map> expected = new HashMap>(); + expected.put(1, Arrays.asList("aa", "bb")); + expected.put(2, Arrays.asList("cccc", "dddd")); + + mapped.subscribe(objectObserver); + + verify(objectObserver, times(1)).onError(any(Throwable.class)); + verify(objectObserver, never()).onNext(expected); + verify(objectObserver, never()).onCompleted(); + } + + @Test + public void testToMultimapWithMapThrowingFactory() { + Observable source = Observable.from("a", "b", "cc", "dd", "eee", "fff"); + + Func0>> mapFactory = new Func0>>() { + @Override + public Map> call() { + throw new RuntimeException("Forced failure"); + } + }; + + Observable>> mapped = Observable.create( + OperationToMultimap.toMultimap(source, lengthFunc, Functions.identity(), mapFactory)); + + Map> expected = new HashMap>(); + expected.put(2, Arrays.asList("cc", "dd")); + expected.put(3, Arrays.asList("eee", "fff")); + + mapped.subscribe(objectObserver); + + verify(objectObserver, times(1)).onError(any(Throwable.class)); + verify(objectObserver, never()).onNext(expected); + verify(objectObserver, never()).onCompleted(); + } + @Test + public void testToMultimapWithThrowingCollectionFactory() { + Observable source = Observable.from("cc", "cc", "eee", "eee"); + + Func1> collectionFactory = new Func1>() { + + @Override + public Collection call(Integer t1) { + if (t1 == 2) { + throw new RuntimeException("Forced failure"); + } else { + return new HashSet(); + } + } + }; + + Observable>> mapped = Observable.create( + OperationToMultimap.toMultimap( + source, lengthFunc, Functions.identity(), new DefaultToMultimapFactory(), collectionFactory)); + + Map> expected = new HashMap>(); + expected.put(2, Arrays.asList("cc", "dd")); + expected.put(3, Collections.singleton("eee")); + + mapped.subscribe(objectObserver); + + verify(objectObserver, times(1)).onError(any(Throwable.class)); + verify(objectObserver, never()).onNext(expected); + verify(objectObserver, never()).onCompleted(); + } +} diff --git a/rxjava-core/src/test/java/rx/subscriptions/SerialSubscriptionTests.java b/rxjava-core/src/test/java/rx/subscriptions/SerialSubscriptionTests.java index 6c8648e986..1569d9c7df 100644 --- a/rxjava-core/src/test/java/rx/subscriptions/SerialSubscriptionTests.java +++ b/rxjava-core/src/test/java/rx/subscriptions/SerialSubscriptionTests.java @@ -1,211 +1,211 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package rx.subscriptions; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.fail; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyNoMoreInteractions; -import static org.mockito.Mockito.verifyZeroInteractions; - -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.CountDownLatch; - -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.runners.MockitoJUnitRunner; - -import rx.Subscription; - -@RunWith(MockitoJUnitRunner.class) -public class SerialSubscriptionTests { - private SerialSubscription serialSubscription; - - @Before - public void setUp() { - serialSubscription = new SerialSubscription(); - } - - @Test - public void unsubscribingWithoutUnderlyingDoesNothing() { - serialSubscription.unsubscribe(); - } - - @Test - public void getSubscriptionShouldReturnSubscriptionAfterUnsubscribe() { - final Subscription underlying = mock(Subscription.class); - serialSubscription.setSubscription(underlying); - serialSubscription.unsubscribe(); - assertEquals(null, serialSubscription.getSubscription()); - } - - @Test - public void getSubscriptionShouldReturnSetSubscription() { - final Subscription underlying = mock(Subscription.class); - serialSubscription.setSubscription(underlying); - assertSame(underlying, serialSubscription.getSubscription()); - - final Subscription another = mock(Subscription.class); - serialSubscription.setSubscription(another); - assertSame(another, serialSubscription.getSubscription()); - } - - @Test - public void unsubscribingTwiceDoesUnsubscribeOnce() { - Subscription underlying = mock(Subscription.class); - serialSubscription.setSubscription(underlying); - - serialSubscription.unsubscribe(); - verify(underlying).unsubscribe(); - - serialSubscription.unsubscribe(); - verifyNoMoreInteractions(underlying); - } - - @Test - public void settingSameSubscriptionTwiceDoesUnsubscribeIt() { - Subscription underlying = mock(Subscription.class); - serialSubscription.setSubscription(underlying); - verifyZeroInteractions(underlying); - serialSubscription.setSubscription(underlying); - verify(underlying).unsubscribe(); - } - - @Test - public void unsubscribingWithSingleUnderlyingUnsubscribes() { - Subscription underlying = mock(Subscription.class); - serialSubscription.setSubscription(underlying); - underlying.unsubscribe(); - verify(underlying).unsubscribe(); - } - - @Test - public void replacingFirstUnderlyingCausesUnsubscription() { - Subscription first = mock(Subscription.class); - serialSubscription.setSubscription(first); - Subscription second = mock(Subscription.class); - serialSubscription.setSubscription(second); - verify(first).unsubscribe(); - } - - @Test - public void whenUnsubscribingSecondUnderlyingUnsubscribed() { - Subscription first = mock(Subscription.class); - serialSubscription.setSubscription(first); - Subscription second = mock(Subscription.class); - serialSubscription.setSubscription(second); - serialSubscription.unsubscribe(); - verify(second).unsubscribe(); - } - - @Test - public void settingUnderlyingWhenUnsubscribedCausesImmediateUnsubscription() { - serialSubscription.unsubscribe(); - Subscription underlying = mock(Subscription.class); - serialSubscription.setSubscription(underlying); - verify(underlying).unsubscribe(); - } - - @Test(timeout = 1000) - public void settingUnderlyingWhenUnsubscribedCausesImmediateUnsubscriptionConcurrently() - throws InterruptedException { - final Subscription firstSet = mock(Subscription.class); - serialSubscription.setSubscription(firstSet); - - final CountDownLatch start = new CountDownLatch(1); - - final int count = 10; - final CountDownLatch end = new CountDownLatch(count); - - final List threads = new ArrayList(); - for (int i = 0 ; i < count ; i++) { - final Thread t = new Thread() { - @Override - public void run() { - try { - start.await(); - serialSubscription.unsubscribe(); - } catch (InterruptedException e) { - fail(e.getMessage()); - } finally { - end.countDown(); - } - } - }; - t.start(); - threads.add(t); - } - - final Subscription underlying = mock(Subscription.class); - start.countDown(); - serialSubscription.setSubscription(underlying); - end.await(); - verify(firstSet).unsubscribe(); - verify(underlying).unsubscribe(); - - for (final Thread t : threads) { - t.join(); - } - } - - @Test - public void concurrentSetSubscriptionShouldNotInterleave() - throws InterruptedException { - final int count = 10; - final List subscriptions = new ArrayList(); - - final CountDownLatch start = new CountDownLatch(1); - final CountDownLatch end = new CountDownLatch(count); - - final List threads = new ArrayList(); - for (int i = 0 ; i < count ; i++) { - final Subscription subscription = mock(Subscription.class); - subscriptions.add(subscription); - - final Thread t = new Thread() { - @Override - public void run() { - try { - start.await(); - serialSubscription.setSubscription(subscription); - } catch (InterruptedException e) { - fail(e.getMessage()); - } finally { - end.countDown(); - } - } - }; - t.start(); - threads.add(t); - } - - start.countDown(); - end.await(); - serialSubscription.unsubscribe(); - - for(final Subscription subscription : subscriptions) { - verify(subscription).unsubscribe(); - } - - for (final Thread t : threads) { - t.join(); - } - } -} +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package rx.subscriptions; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.fail; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.verifyZeroInteractions; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CountDownLatch; + +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.runners.MockitoJUnitRunner; + +import rx.Subscription; + +@RunWith(MockitoJUnitRunner.class) +public class SerialSubscriptionTests { + private SerialSubscription serialSubscription; + + @Before + public void setUp() { + serialSubscription = new SerialSubscription(); + } + + @Test + public void unsubscribingWithoutUnderlyingDoesNothing() { + serialSubscription.unsubscribe(); + } + + @Test + public void getSubscriptionShouldReturnSubscriptionAfterUnsubscribe() { + final Subscription underlying = mock(Subscription.class); + serialSubscription.setSubscription(underlying); + serialSubscription.unsubscribe(); + assertEquals(null, serialSubscription.getSubscription()); + } + + @Test + public void getSubscriptionShouldReturnSetSubscription() { + final Subscription underlying = mock(Subscription.class); + serialSubscription.setSubscription(underlying); + assertSame(underlying, serialSubscription.getSubscription()); + + final Subscription another = mock(Subscription.class); + serialSubscription.setSubscription(another); + assertSame(another, serialSubscription.getSubscription()); + } + + @Test + public void unsubscribingTwiceDoesUnsubscribeOnce() { + Subscription underlying = mock(Subscription.class); + serialSubscription.setSubscription(underlying); + + serialSubscription.unsubscribe(); + verify(underlying).unsubscribe(); + + serialSubscription.unsubscribe(); + verifyNoMoreInteractions(underlying); + } + + @Test + public void settingSameSubscriptionTwiceDoesUnsubscribeIt() { + Subscription underlying = mock(Subscription.class); + serialSubscription.setSubscription(underlying); + verifyZeroInteractions(underlying); + serialSubscription.setSubscription(underlying); + verify(underlying).unsubscribe(); + } + + @Test + public void unsubscribingWithSingleUnderlyingUnsubscribes() { + Subscription underlying = mock(Subscription.class); + serialSubscription.setSubscription(underlying); + underlying.unsubscribe(); + verify(underlying).unsubscribe(); + } + + @Test + public void replacingFirstUnderlyingCausesUnsubscription() { + Subscription first = mock(Subscription.class); + serialSubscription.setSubscription(first); + Subscription second = mock(Subscription.class); + serialSubscription.setSubscription(second); + verify(first).unsubscribe(); + } + + @Test + public void whenUnsubscribingSecondUnderlyingUnsubscribed() { + Subscription first = mock(Subscription.class); + serialSubscription.setSubscription(first); + Subscription second = mock(Subscription.class); + serialSubscription.setSubscription(second); + serialSubscription.unsubscribe(); + verify(second).unsubscribe(); + } + + @Test + public void settingUnderlyingWhenUnsubscribedCausesImmediateUnsubscription() { + serialSubscription.unsubscribe(); + Subscription underlying = mock(Subscription.class); + serialSubscription.setSubscription(underlying); + verify(underlying).unsubscribe(); + } + + @Test(timeout = 1000) + public void settingUnderlyingWhenUnsubscribedCausesImmediateUnsubscriptionConcurrently() + throws InterruptedException { + final Subscription firstSet = mock(Subscription.class); + serialSubscription.setSubscription(firstSet); + + final CountDownLatch start = new CountDownLatch(1); + + final int count = 10; + final CountDownLatch end = new CountDownLatch(count); + + final List threads = new ArrayList(); + for (int i = 0 ; i < count ; i++) { + final Thread t = new Thread() { + @Override + public void run() { + try { + start.await(); + serialSubscription.unsubscribe(); + } catch (InterruptedException e) { + fail(e.getMessage()); + } finally { + end.countDown(); + } + } + }; + t.start(); + threads.add(t); + } + + final Subscription underlying = mock(Subscription.class); + start.countDown(); + serialSubscription.setSubscription(underlying); + end.await(); + verify(firstSet).unsubscribe(); + verify(underlying).unsubscribe(); + + for (final Thread t : threads) { + t.join(); + } + } + + @Test + public void concurrentSetSubscriptionShouldNotInterleave() + throws InterruptedException { + final int count = 10; + final List subscriptions = new ArrayList(); + + final CountDownLatch start = new CountDownLatch(1); + final CountDownLatch end = new CountDownLatch(count); + + final List threads = new ArrayList(); + for (int i = 0 ; i < count ; i++) { + final Subscription subscription = mock(Subscription.class); + subscriptions.add(subscription); + + final Thread t = new Thread() { + @Override + public void run() { + try { + start.await(); + serialSubscription.setSubscription(subscription); + } catch (InterruptedException e) { + fail(e.getMessage()); + } finally { + end.countDown(); + } + } + }; + t.start(); + threads.add(t); + } + + start.countDown(); + end.await(); + serialSubscription.unsubscribe(); + + for(final Subscription subscription : subscriptions) { + verify(subscription).unsubscribe(); + } + + for (final Thread t : threads) { + t.join(); + } + } +} diff --git a/rxjava-core/src/test/java/rx/util/functions/AsyncTest.java b/rxjava-core/src/test/java/rx/util/functions/AsyncTest.java index 5b66dcf82d..cc76170894 100644 --- a/rxjava-core/src/test/java/rx/util/functions/AsyncTest.java +++ b/rxjava-core/src/test/java/rx/util/functions/AsyncTest.java @@ -1,653 +1,653 @@ -/** - * Copyright 2013 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package rx.util.functions; - -import java.util.concurrent.atomic.AtomicInteger; -import junit.framework.Assert; -import org.junit.Before; -import org.junit.Test; -import static org.mockito.Matchers.any; -import org.mockito.Mock; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import org.mockito.MockitoAnnotations; -import rx.Observer; -import rx.concurrency.Schedulers; -import rx.util.functions.Action0; -import rx.util.functions.Action1; -import rx.util.functions.Action2; -import rx.util.functions.Action3; -import rx.util.functions.Action4; -import rx.util.functions.Action5; -import rx.util.functions.Action6; -import rx.util.functions.Action7; -import rx.util.functions.Action8; -import rx.util.functions.Action9; -import rx.util.functions.ActionN; -import rx.util.functions.Func0; -import rx.util.functions.Func1; -import rx.util.functions.Func2; -import rx.util.functions.Func3; -import rx.util.functions.Func4; -import rx.util.functions.Func5; -import rx.util.functions.Func6; -import rx.util.functions.Func7; -import rx.util.functions.Func8; -import rx.util.functions.Func9; -import rx.util.functions.FuncN; - -public class AsyncTest { - @Mock - Observer observer; - @Before - public void before() { - MockitoAnnotations.initMocks(this); - } - @Test - public void testAction0() { - final AtomicInteger value = new AtomicInteger(); - Action0 action = new Action0() { - @Override - public void call() { - value.incrementAndGet(); - } - }; - - Async.toAsync(action, Schedulers.immediate()) - .call() - .subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(null); - verify(observer, times(1)).onCompleted(); - - Assert.assertEquals(1, value.get()); - } - @Test - public void testAction0Error() { - Action0 action = new Action0() { - @Override - public void call() { - throw new RuntimeException("Forced failure"); - } - }; - - Async.toAsync(action, Schedulers.immediate()) - .call() - .subscribe(observer); - - verify(observer, times(1)).onError(any(Throwable.class)); - verify(observer, never()).onNext(null); - verify(observer, never()).onCompleted(); - } - @Test - public void testAction1() { - final AtomicInteger value = new AtomicInteger(); - Action1 action = new Action1() { - @Override - public void call(Integer t1) { - value.set(t1); - } - }; - - Async.toAsync(action, Schedulers.immediate()) - .call(1) - .subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(null); - verify(observer, times(1)).onCompleted(); - - Assert.assertEquals(1, value.get()); - } - @Test - public void testAction1Error() { - Action1 action = new Action1() { - @Override - public void call(Integer t1) { - throw new RuntimeException("Forced failure"); - } - }; - - Async.toAsync(action, Schedulers.immediate()) - .call(1) - .subscribe(observer); - - verify(observer, times(1)).onError(any(Throwable.class)); - verify(observer, never()).onNext(null); - verify(observer, never()).onCompleted(); - } - @Test - public void testAction2() { - final AtomicInteger value = new AtomicInteger(); - Action2 action = new Action2() { - @Override - public void call(Integer t1, Integer t2) { - value.set(t1 | t2); - } - }; - - Async.toAsync(action, Schedulers.immediate()) - .call(1, 2) - .subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(null); - verify(observer, times(1)).onCompleted(); - - Assert.assertEquals(3, value.get()); - } - @Test - public void testAction2Error() { - Action2 action = new Action2() { - @Override - public void call(Integer t1, Integer t2) { - throw new RuntimeException("Forced failure"); - } - }; - - Async.toAsync(action, Schedulers.immediate()) - .call(1, 2) - .subscribe(observer); - - verify(observer, times(1)).onError(any(Throwable.class)); - verify(observer, never()).onNext(null); - verify(observer, never()).onCompleted(); - } - @Test - public void testAction3() { - final AtomicInteger value = new AtomicInteger(); - Action3 action = new Action3() { - @Override - public void call(Integer t1, Integer t2, Integer t3) { - value.set(t1 | t2 | t3); - } - }; - - Async.toAsync(action, Schedulers.immediate()) - .call(1, 2, 4) - .subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(null); - verify(observer, times(1)).onCompleted(); - - Assert.assertEquals(7, value.get()); - } - @Test - public void testAction3Error() { - Action3 action = new Action3() { - @Override - public void call(Integer t1, Integer t2, Integer t3) { - throw new RuntimeException("Forced failure"); - } - }; - - Async.toAsync(action, Schedulers.immediate()) - .call(1, 2, 4) - .subscribe(observer); - - verify(observer, times(1)).onError(any(Throwable.class)); - verify(observer, never()).onNext(null); - verify(observer, never()).onCompleted(); - } - @Test - public void testAction4() { - final AtomicInteger value = new AtomicInteger(); - Action4 action = new Action4() { - @Override - public void call(Integer t1, Integer t2, Integer t3, Integer t4) { - value.set(t1 | t2 | t3 | t4); - } - }; - - Async.toAsync(action, Schedulers.immediate()) - .call(1, 2, 4, 8) - .subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(null); - verify(observer, times(1)).onCompleted(); - - Assert.assertEquals(15, value.get()); - } - @Test - public void testAction4Error() { - Action4 action = new Action4() { - @Override - public void call(Integer t1, Integer t2, Integer t3, Integer t4) { - throw new RuntimeException("Forced failure"); - } - }; - - Async.toAsync(action, Schedulers.immediate()) - .call(1, 2, 4, 8) - .subscribe(observer); - - verify(observer, times(1)).onError(any(Throwable.class)); - verify(observer, never()).onNext(null); - verify(observer, never()).onCompleted(); - } - @Test - public void testAction5() { - final AtomicInteger value = new AtomicInteger(); - Action5 action = new Action5() { - @Override - public void call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5) { - value.set(t1 | t2 | t3 | t4 | t5); - } - }; - - Async.toAsync(action, Schedulers.immediate()) - .call(1, 2, 4, 8, 16) - .subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(null); - verify(observer, times(1)).onCompleted(); - - Assert.assertEquals(31, value.get()); - } - @Test - public void testAction5Error() { - Action5 action = new Action5() { - @Override - public void call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5) { - throw new RuntimeException("Forced failure"); - } - }; - - Async.toAsync(action, Schedulers.immediate()) - .call(1, 2, 4, 8, 16) - .subscribe(observer); - - verify(observer, times(1)).onError(any(Throwable.class)); - verify(observer, never()).onNext(null); - verify(observer, never()).onCompleted(); - } - @Test - public void testAction6() { - final AtomicInteger value = new AtomicInteger(); - Action6 action = new Action6() { - @Override - public void call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6) { - value.set(t1 | t2 | t3 | t4 | t5 | t6); - } - }; - - Async.toAsync(action, Schedulers.immediate()) - .call(1, 2, 4, 8, 16, 32) - .subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(null); - verify(observer, times(1)).onCompleted(); - - Assert.assertEquals(63, value.get()); - } - @Test - public void testAction6Error() { - Action6 action = new Action6() { - @Override - public void call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6) { - throw new RuntimeException("Forced failure"); - } - }; - - Async.toAsync(action, Schedulers.immediate()) - .call(1, 2, 4, 8, 16, 32) - .subscribe(observer); - - verify(observer, times(1)).onError(any(Throwable.class)); - verify(observer, never()).onNext(null); - verify(observer, never()).onCompleted(); - } - @Test - public void testAction7() { - final AtomicInteger value = new AtomicInteger(); - Action7 action = new Action7() { - @Override - public void call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6, Integer t7) { - value.set(t1 | t2 | t3 | t4 | t5 | t6 | t7); - } - }; - - Async.toAsync(action, Schedulers.immediate()) - .call(1, 2, 4, 8, 16, 32, 64) - .subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(null); - verify(observer, times(1)).onCompleted(); - - Assert.assertEquals(127, value.get()); - } - @Test - public void testAction7Error() { - Action7 action = new Action7() { - @Override - public void call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6, Integer t7) { - throw new RuntimeException("Forced failure"); - } - }; - - Async.toAsync(action, Schedulers.immediate()) - .call(1, 2, 4, 8, 16, 32, 64) - .subscribe(observer); - - verify(observer, times(1)).onError(any(Throwable.class)); - verify(observer, never()).onNext(null); - verify(observer, never()).onCompleted(); - } - @Test - public void testAction8() { - final AtomicInteger value = new AtomicInteger(); - Action8 action = new Action8() { - @Override - public void call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6, Integer t7, Integer t8) { - value.set(t1 | t2 | t3 | t4 | t5 | t6 | t7 | t8); - } - }; - - Async.toAsync(action, Schedulers.immediate()) - .call(1, 2, 4, 8, 16, 32, 64, 128) - .subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(null); - verify(observer, times(1)).onCompleted(); - - Assert.assertEquals(255, value.get()); - } - @Test - public void testAction8Error() { - Action8 action = new Action8() { - @Override - public void call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6, Integer t7, Integer t8) { - throw new RuntimeException("Forced failure"); - } - }; - - Async.toAsync(action, Schedulers.immediate()) - .call(1, 2, 4, 8, 16, 32, 64, 128) - .subscribe(observer); - - verify(observer, times(1)).onError(any(Throwable.class)); - verify(observer, never()).onNext(null); - verify(observer, never()).onCompleted(); - } - @Test - public void testAction9() { - final AtomicInteger value = new AtomicInteger(); - Action9 action = new Action9() { - @Override - public void call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6, Integer t7, Integer t8, Integer t9) { - value.set(t1 | t2 | t3 | t4 | t5 | t6 | t7 | t8 | t9); - } - }; - - Async.toAsync(action, Schedulers.immediate()) - .call(1, 2, 4, 8, 16, 32, 64, 128, 256) - .subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(null); - verify(observer, times(1)).onCompleted(); - - Assert.assertEquals(511, value.get()); - } - @Test - public void testAction9Error() { - Action9 action = new Action9() { - @Override - public void call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6, Integer t7, Integer t8, Integer t9) { - throw new RuntimeException("Forced failure"); - } - }; - - Async.toAsync(action, Schedulers.immediate()) - .call(1, 2, 4, 8, 16, 32, 64, 128, 256) - .subscribe(observer); - - verify(observer, times(1)).onError(any(Throwable.class)); - verify(observer, never()).onNext(null); - verify(observer, never()).onCompleted(); - } - @Test - public void testActionN() { - final AtomicInteger value = new AtomicInteger(); - ActionN action = new ActionN() { - @Override - public void call(Object... args) { - int i = 0; - for (Object o : args) { - i = i | (Integer)o; - } - value.set(i); - } - }; - - Async.toAsync(action, Schedulers.immediate()) - .call(1, 2, 4, 8, 16, 32, 64, 128, 256, 512) - .subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(null); - verify(observer, times(1)).onCompleted(); - - Assert.assertEquals(1023, value.get()); - } - @Test - public void testActionNError() { - ActionN action = new ActionN() { - @Override - public void call(Object... args) { - throw new RuntimeException("Forced failure"); - } - }; - - Async.toAsync(action, Schedulers.immediate()) - .call(1, 2, 4, 8, 16, 32, 64, 128, 256, 512) - .subscribe(observer); - - verify(observer, times(1)).onError(any(Throwable.class)); - verify(observer, never()).onNext(null); - verify(observer, never()).onCompleted(); - } - @Test - public void testFunc0() { - Func0 func = new Func0() { - @Override - public Integer call() { - return 0; - } - }; - Async.toAsync(func, Schedulers.immediate()) - .call() - .subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(0); - verify(observer, times(1)).onCompleted(); - - } - @Test - public void testFunc1() { - Func1 func = new Func1() { - @Override - public Integer call(Integer t1) { - return t1; - } - }; - Async.toAsync(func, Schedulers.immediate()) - .call(1) - .subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(1); - verify(observer, times(1)).onCompleted(); - } - @Test - public void testFunc2() { - Func2 func = new Func2() { - @Override - public Integer call(Integer t1, Integer t2) { - return t1 | t2; - } - }; - Async.toAsync(func, Schedulers.immediate()) - .call(1, 2) - .subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(3); - verify(observer, times(1)).onCompleted(); - } - @Test - public void testFunc3() { - Func3 func = new Func3() { - @Override - public Integer call(Integer t1, Integer t2, Integer t3) { - return t1 | t2 | t3; - } - }; - Async.toAsync(func, Schedulers.immediate()) - .call(1, 2, 4) - .subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(7); - verify(observer, times(1)).onCompleted(); - } - @Test - public void testFunc4() { - Func4 func = new Func4() { - @Override - public Integer call(Integer t1, Integer t2, Integer t3, Integer t4) { - return t1 | t2 | t3 | t4; - } - }; - Async.toAsync(func, Schedulers.immediate()) - .call(1, 2, 4, 8) - .subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(15); - verify(observer, times(1)).onCompleted(); - } - @Test - public void testFunc5() { - Func5 func = new Func5() { - @Override - public Integer call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5) { - return t1 | t2 | t3 | t4 | t5; - } - }; - Async.toAsync(func, Schedulers.immediate()) - .call(1, 2, 4, 8, 16) - .subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(31); - verify(observer, times(1)).onCompleted(); - } - @Test - public void testFunc6() { - Func6 func = new Func6() { - @Override - public Integer call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6) { - return t1 | t2 | t3 | t4 | t5 | t6; - } - }; - Async.toAsync(func, Schedulers.immediate()) - .call(1, 2, 4, 8, 16, 32) - .subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(63); - verify(observer, times(1)).onCompleted(); - } - @Test - public void testFunc7() { - Func7 func = new Func7() { - @Override - public Integer call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6, Integer t7) { - return t1 | t2 | t3 | t4 | t5 | t6 | t7; - } - }; - Async.toAsync(func, Schedulers.immediate()) - .call(1, 2, 4, 8, 16, 32, 64) - .subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(127); - verify(observer, times(1)).onCompleted(); - } - @Test - public void testFunc8() { - Func8 func = new Func8() { - @Override - public Integer call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6, Integer t7, Integer t8) { - return t1 | t2 | t3 | t4 | t5 | t6 | t7 | t8; - } - }; - Async.toAsync(func, Schedulers.immediate()) - .call(1, 2, 4, 8, 16, 32, 64, 128) - .subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(255); - verify(observer, times(1)).onCompleted(); - } - @Test - public void testFunc9() { - Func9 func = new Func9() { - @Override - public Integer call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6, Integer t7, Integer t8, Integer t9) { - return t1 | t2 | t3 | t4 | t5 | t6 | t7 | t8 | t9; - } - }; - Async.toAsync(func, Schedulers.immediate()) - .call(1, 2, 4, 8, 16, 32, 64, 128, 256) - .subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(511); - verify(observer, times(1)).onCompleted(); - } - @Test - public void testFuncN() { - FuncN func = new FuncN() { - @Override - public Integer call(Object... args) { - int i = 0; - for (Object o : args) { - i = i | (Integer)o; - } - return i; - } - }; - Async.toAsync(func, Schedulers.immediate()) - .call(1, 2, 4, 8, 16, 32, 64, 128, 256, 512) - .subscribe(observer); - - verify(observer, never()).onError(any(Throwable.class)); - verify(observer, times(1)).onNext(1023); - verify(observer, times(1)).onCompleted(); - } -} +/** + * Copyright 2013 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package rx.util.functions; + +import java.util.concurrent.atomic.AtomicInteger; +import junit.framework.Assert; +import org.junit.Before; +import org.junit.Test; +import static org.mockito.Matchers.any; +import org.mockito.Mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import org.mockito.MockitoAnnotations; +import rx.Observer; +import rx.concurrency.Schedulers; +import rx.util.functions.Action0; +import rx.util.functions.Action1; +import rx.util.functions.Action2; +import rx.util.functions.Action3; +import rx.util.functions.Action4; +import rx.util.functions.Action5; +import rx.util.functions.Action6; +import rx.util.functions.Action7; +import rx.util.functions.Action8; +import rx.util.functions.Action9; +import rx.util.functions.ActionN; +import rx.util.functions.Func0; +import rx.util.functions.Func1; +import rx.util.functions.Func2; +import rx.util.functions.Func3; +import rx.util.functions.Func4; +import rx.util.functions.Func5; +import rx.util.functions.Func6; +import rx.util.functions.Func7; +import rx.util.functions.Func8; +import rx.util.functions.Func9; +import rx.util.functions.FuncN; + +public class AsyncTest { + @Mock + Observer observer; + @Before + public void before() { + MockitoAnnotations.initMocks(this); + } + @Test + public void testAction0() { + final AtomicInteger value = new AtomicInteger(); + Action0 action = new Action0() { + @Override + public void call() { + value.incrementAndGet(); + } + }; + + Async.toAsync(action, Schedulers.immediate()) + .call() + .subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(null); + verify(observer, times(1)).onCompleted(); + + Assert.assertEquals(1, value.get()); + } + @Test + public void testAction0Error() { + Action0 action = new Action0() { + @Override + public void call() { + throw new RuntimeException("Forced failure"); + } + }; + + Async.toAsync(action, Schedulers.immediate()) + .call() + .subscribe(observer); + + verify(observer, times(1)).onError(any(Throwable.class)); + verify(observer, never()).onNext(null); + verify(observer, never()).onCompleted(); + } + @Test + public void testAction1() { + final AtomicInteger value = new AtomicInteger(); + Action1 action = new Action1() { + @Override + public void call(Integer t1) { + value.set(t1); + } + }; + + Async.toAsync(action, Schedulers.immediate()) + .call(1) + .subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(null); + verify(observer, times(1)).onCompleted(); + + Assert.assertEquals(1, value.get()); + } + @Test + public void testAction1Error() { + Action1 action = new Action1() { + @Override + public void call(Integer t1) { + throw new RuntimeException("Forced failure"); + } + }; + + Async.toAsync(action, Schedulers.immediate()) + .call(1) + .subscribe(observer); + + verify(observer, times(1)).onError(any(Throwable.class)); + verify(observer, never()).onNext(null); + verify(observer, never()).onCompleted(); + } + @Test + public void testAction2() { + final AtomicInteger value = new AtomicInteger(); + Action2 action = new Action2() { + @Override + public void call(Integer t1, Integer t2) { + value.set(t1 | t2); + } + }; + + Async.toAsync(action, Schedulers.immediate()) + .call(1, 2) + .subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(null); + verify(observer, times(1)).onCompleted(); + + Assert.assertEquals(3, value.get()); + } + @Test + public void testAction2Error() { + Action2 action = new Action2() { + @Override + public void call(Integer t1, Integer t2) { + throw new RuntimeException("Forced failure"); + } + }; + + Async.toAsync(action, Schedulers.immediate()) + .call(1, 2) + .subscribe(observer); + + verify(observer, times(1)).onError(any(Throwable.class)); + verify(observer, never()).onNext(null); + verify(observer, never()).onCompleted(); + } + @Test + public void testAction3() { + final AtomicInteger value = new AtomicInteger(); + Action3 action = new Action3() { + @Override + public void call(Integer t1, Integer t2, Integer t3) { + value.set(t1 | t2 | t3); + } + }; + + Async.toAsync(action, Schedulers.immediate()) + .call(1, 2, 4) + .subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(null); + verify(observer, times(1)).onCompleted(); + + Assert.assertEquals(7, value.get()); + } + @Test + public void testAction3Error() { + Action3 action = new Action3() { + @Override + public void call(Integer t1, Integer t2, Integer t3) { + throw new RuntimeException("Forced failure"); + } + }; + + Async.toAsync(action, Schedulers.immediate()) + .call(1, 2, 4) + .subscribe(observer); + + verify(observer, times(1)).onError(any(Throwable.class)); + verify(observer, never()).onNext(null); + verify(observer, never()).onCompleted(); + } + @Test + public void testAction4() { + final AtomicInteger value = new AtomicInteger(); + Action4 action = new Action4() { + @Override + public void call(Integer t1, Integer t2, Integer t3, Integer t4) { + value.set(t1 | t2 | t3 | t4); + } + }; + + Async.toAsync(action, Schedulers.immediate()) + .call(1, 2, 4, 8) + .subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(null); + verify(observer, times(1)).onCompleted(); + + Assert.assertEquals(15, value.get()); + } + @Test + public void testAction4Error() { + Action4 action = new Action4() { + @Override + public void call(Integer t1, Integer t2, Integer t3, Integer t4) { + throw new RuntimeException("Forced failure"); + } + }; + + Async.toAsync(action, Schedulers.immediate()) + .call(1, 2, 4, 8) + .subscribe(observer); + + verify(observer, times(1)).onError(any(Throwable.class)); + verify(observer, never()).onNext(null); + verify(observer, never()).onCompleted(); + } + @Test + public void testAction5() { + final AtomicInteger value = new AtomicInteger(); + Action5 action = new Action5() { + @Override + public void call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5) { + value.set(t1 | t2 | t3 | t4 | t5); + } + }; + + Async.toAsync(action, Schedulers.immediate()) + .call(1, 2, 4, 8, 16) + .subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(null); + verify(observer, times(1)).onCompleted(); + + Assert.assertEquals(31, value.get()); + } + @Test + public void testAction5Error() { + Action5 action = new Action5() { + @Override + public void call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5) { + throw new RuntimeException("Forced failure"); + } + }; + + Async.toAsync(action, Schedulers.immediate()) + .call(1, 2, 4, 8, 16) + .subscribe(observer); + + verify(observer, times(1)).onError(any(Throwable.class)); + verify(observer, never()).onNext(null); + verify(observer, never()).onCompleted(); + } + @Test + public void testAction6() { + final AtomicInteger value = new AtomicInteger(); + Action6 action = new Action6() { + @Override + public void call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6) { + value.set(t1 | t2 | t3 | t4 | t5 | t6); + } + }; + + Async.toAsync(action, Schedulers.immediate()) + .call(1, 2, 4, 8, 16, 32) + .subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(null); + verify(observer, times(1)).onCompleted(); + + Assert.assertEquals(63, value.get()); + } + @Test + public void testAction6Error() { + Action6 action = new Action6() { + @Override + public void call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6) { + throw new RuntimeException("Forced failure"); + } + }; + + Async.toAsync(action, Schedulers.immediate()) + .call(1, 2, 4, 8, 16, 32) + .subscribe(observer); + + verify(observer, times(1)).onError(any(Throwable.class)); + verify(observer, never()).onNext(null); + verify(observer, never()).onCompleted(); + } + @Test + public void testAction7() { + final AtomicInteger value = new AtomicInteger(); + Action7 action = new Action7() { + @Override + public void call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6, Integer t7) { + value.set(t1 | t2 | t3 | t4 | t5 | t6 | t7); + } + }; + + Async.toAsync(action, Schedulers.immediate()) + .call(1, 2, 4, 8, 16, 32, 64) + .subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(null); + verify(observer, times(1)).onCompleted(); + + Assert.assertEquals(127, value.get()); + } + @Test + public void testAction7Error() { + Action7 action = new Action7() { + @Override + public void call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6, Integer t7) { + throw new RuntimeException("Forced failure"); + } + }; + + Async.toAsync(action, Schedulers.immediate()) + .call(1, 2, 4, 8, 16, 32, 64) + .subscribe(observer); + + verify(observer, times(1)).onError(any(Throwable.class)); + verify(observer, never()).onNext(null); + verify(observer, never()).onCompleted(); + } + @Test + public void testAction8() { + final AtomicInteger value = new AtomicInteger(); + Action8 action = new Action8() { + @Override + public void call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6, Integer t7, Integer t8) { + value.set(t1 | t2 | t3 | t4 | t5 | t6 | t7 | t8); + } + }; + + Async.toAsync(action, Schedulers.immediate()) + .call(1, 2, 4, 8, 16, 32, 64, 128) + .subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(null); + verify(observer, times(1)).onCompleted(); + + Assert.assertEquals(255, value.get()); + } + @Test + public void testAction8Error() { + Action8 action = new Action8() { + @Override + public void call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6, Integer t7, Integer t8) { + throw new RuntimeException("Forced failure"); + } + }; + + Async.toAsync(action, Schedulers.immediate()) + .call(1, 2, 4, 8, 16, 32, 64, 128) + .subscribe(observer); + + verify(observer, times(1)).onError(any(Throwable.class)); + verify(observer, never()).onNext(null); + verify(observer, never()).onCompleted(); + } + @Test + public void testAction9() { + final AtomicInteger value = new AtomicInteger(); + Action9 action = new Action9() { + @Override + public void call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6, Integer t7, Integer t8, Integer t9) { + value.set(t1 | t2 | t3 | t4 | t5 | t6 | t7 | t8 | t9); + } + }; + + Async.toAsync(action, Schedulers.immediate()) + .call(1, 2, 4, 8, 16, 32, 64, 128, 256) + .subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(null); + verify(observer, times(1)).onCompleted(); + + Assert.assertEquals(511, value.get()); + } + @Test + public void testAction9Error() { + Action9 action = new Action9() { + @Override + public void call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6, Integer t7, Integer t8, Integer t9) { + throw new RuntimeException("Forced failure"); + } + }; + + Async.toAsync(action, Schedulers.immediate()) + .call(1, 2, 4, 8, 16, 32, 64, 128, 256) + .subscribe(observer); + + verify(observer, times(1)).onError(any(Throwable.class)); + verify(observer, never()).onNext(null); + verify(observer, never()).onCompleted(); + } + @Test + public void testActionN() { + final AtomicInteger value = new AtomicInteger(); + ActionN action = new ActionN() { + @Override + public void call(Object... args) { + int i = 0; + for (Object o : args) { + i = i | (Integer)o; + } + value.set(i); + } + }; + + Async.toAsync(action, Schedulers.immediate()) + .call(1, 2, 4, 8, 16, 32, 64, 128, 256, 512) + .subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(null); + verify(observer, times(1)).onCompleted(); + + Assert.assertEquals(1023, value.get()); + } + @Test + public void testActionNError() { + ActionN action = new ActionN() { + @Override + public void call(Object... args) { + throw new RuntimeException("Forced failure"); + } + }; + + Async.toAsync(action, Schedulers.immediate()) + .call(1, 2, 4, 8, 16, 32, 64, 128, 256, 512) + .subscribe(observer); + + verify(observer, times(1)).onError(any(Throwable.class)); + verify(observer, never()).onNext(null); + verify(observer, never()).onCompleted(); + } + @Test + public void testFunc0() { + Func0 func = new Func0() { + @Override + public Integer call() { + return 0; + } + }; + Async.toAsync(func, Schedulers.immediate()) + .call() + .subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(0); + verify(observer, times(1)).onCompleted(); + + } + @Test + public void testFunc1() { + Func1 func = new Func1() { + @Override + public Integer call(Integer t1) { + return t1; + } + }; + Async.toAsync(func, Schedulers.immediate()) + .call(1) + .subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(1); + verify(observer, times(1)).onCompleted(); + } + @Test + public void testFunc2() { + Func2 func = new Func2() { + @Override + public Integer call(Integer t1, Integer t2) { + return t1 | t2; + } + }; + Async.toAsync(func, Schedulers.immediate()) + .call(1, 2) + .subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(3); + verify(observer, times(1)).onCompleted(); + } + @Test + public void testFunc3() { + Func3 func = new Func3() { + @Override + public Integer call(Integer t1, Integer t2, Integer t3) { + return t1 | t2 | t3; + } + }; + Async.toAsync(func, Schedulers.immediate()) + .call(1, 2, 4) + .subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(7); + verify(observer, times(1)).onCompleted(); + } + @Test + public void testFunc4() { + Func4 func = new Func4() { + @Override + public Integer call(Integer t1, Integer t2, Integer t3, Integer t4) { + return t1 | t2 | t3 | t4; + } + }; + Async.toAsync(func, Schedulers.immediate()) + .call(1, 2, 4, 8) + .subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(15); + verify(observer, times(1)).onCompleted(); + } + @Test + public void testFunc5() { + Func5 func = new Func5() { + @Override + public Integer call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5) { + return t1 | t2 | t3 | t4 | t5; + } + }; + Async.toAsync(func, Schedulers.immediate()) + .call(1, 2, 4, 8, 16) + .subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(31); + verify(observer, times(1)).onCompleted(); + } + @Test + public void testFunc6() { + Func6 func = new Func6() { + @Override + public Integer call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6) { + return t1 | t2 | t3 | t4 | t5 | t6; + } + }; + Async.toAsync(func, Schedulers.immediate()) + .call(1, 2, 4, 8, 16, 32) + .subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(63); + verify(observer, times(1)).onCompleted(); + } + @Test + public void testFunc7() { + Func7 func = new Func7() { + @Override + public Integer call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6, Integer t7) { + return t1 | t2 | t3 | t4 | t5 | t6 | t7; + } + }; + Async.toAsync(func, Schedulers.immediate()) + .call(1, 2, 4, 8, 16, 32, 64) + .subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(127); + verify(observer, times(1)).onCompleted(); + } + @Test + public void testFunc8() { + Func8 func = new Func8() { + @Override + public Integer call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6, Integer t7, Integer t8) { + return t1 | t2 | t3 | t4 | t5 | t6 | t7 | t8; + } + }; + Async.toAsync(func, Schedulers.immediate()) + .call(1, 2, 4, 8, 16, 32, 64, 128) + .subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(255); + verify(observer, times(1)).onCompleted(); + } + @Test + public void testFunc9() { + Func9 func = new Func9() { + @Override + public Integer call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6, Integer t7, Integer t8, Integer t9) { + return t1 | t2 | t3 | t4 | t5 | t6 | t7 | t8 | t9; + } + }; + Async.toAsync(func, Schedulers.immediate()) + .call(1, 2, 4, 8, 16, 32, 64, 128, 256) + .subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(511); + verify(observer, times(1)).onCompleted(); + } + @Test + public void testFuncN() { + FuncN func = new FuncN() { + @Override + public Integer call(Object... args) { + int i = 0; + for (Object o : args) { + i = i | (Integer)o; + } + return i; + } + }; + Async.toAsync(func, Schedulers.immediate()) + .call(1, 2, 4, 8, 16, 32, 64, 128, 256, 512) + .subscribe(observer); + + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, times(1)).onNext(1023); + verify(observer, times(1)).onCompleted(); + } +}