From 5789894638a62ac17b5276053e3bea8bdd570580 Mon Sep 17 00:00:00 2001 From: Bob Cotton Date: Sun, 1 Sep 2013 22:14:41 -0600 Subject: [PATCH 1/5] Merge window operator commit to master This is from commit http://github.com/RallySoftware/RxJava/commit/71335e4ed1531b1542c7b304a6d3337145e0b2df --- rxjava-core/src/main/java/rx/Observable.java | 695 ++++++++-------- .../java/rx/operators/ChunkedOperation.java | 587 ++++++++++++++ .../java/rx/operators/OperationBuffer.java | 754 +++--------------- .../java/rx/operators/OperationWindow.java | 660 +++++++++++++++ .../util/{BufferClosing.java => Closing.java} | 2 +- .../{BufferClosings.java => Closings.java} | 8 +- .../util/{BufferOpening.java => Opening.java} | 2 +- .../{BufferOpenings.java => Openings.java} | 8 +- 8 files changed, 1715 insertions(+), 1001 deletions(-) create mode 100644 rxjava-core/src/main/java/rx/operators/ChunkedOperation.java create mode 100644 rxjava-core/src/main/java/rx/operators/OperationWindow.java rename rxjava-core/src/main/java/rx/util/{BufferClosing.java => Closing.java} (95%) rename rxjava-core/src/main/java/rx/util/{BufferClosings.java => Closings.java} (82%) rename rxjava-core/src/main/java/rx/util/{BufferOpening.java => Opening.java} (95%) rename rxjava-core/src/main/java/rx/util/{BufferOpenings.java => Openings.java} (82%) diff --git a/rxjava-core/src/main/java/rx/Observable.java b/rxjava-core/src/main/java/rx/Observable.java index dad2ca7286..17b54c7039 100644 --- a/rxjava-core/src/main/java/rx/Observable.java +++ b/rxjava-core/src/main/java/rx/Observable.java @@ -61,6 +61,7 @@ import rx.operators.OperationToObservableIterable; import rx.operators.OperationToObservableList; import rx.operators.OperationToObservableSortedList; +import rx.operators.OperationWindow; import rx.operators.OperationZip; import rx.operators.SafeObservableSubscription; import rx.operators.SafeObserver; @@ -71,8 +72,8 @@ import rx.subjects.ReplaySubject; import rx.subjects.Subject; import rx.subscriptions.Subscriptions; -import rx.util.BufferClosing; -import rx.util.BufferOpening; +import rx.util.Closing; +import rx.util.Opening; import rx.util.OnErrorNotImplementedException; import rx.util.Range; import rx.util.Timestamped; @@ -83,11 +84,6 @@ 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; import rx.util.functions.Function; @@ -102,27 +98,18 @@ *

* *

- * For more information see the RxJava Wiki + * For more information see the RxJava + * Wiki * * @param */ public class Observable { - /** - * Executed when 'subscribe' is invoked. - */ - private final OnSubscribeFunc onSubscribe; - - /** - * Function interface for work to be performed when an {@link Observable} is subscribed to via {@link Observable#subscribe(Observer)} - * - * @param - */ - public static interface OnSubscribeFunc extends Function { + //TODO use a consistent parameter naming scheme (for example: for all operators that modify a source Observable, the parameter representing that source Observable should have the same name, e.g. "source" -- currently such parameters are named any of "sequence", "that", "source", "items", or "observable") - public Subscription onSubscribe(Observer t1); + private final static RxJavaObservableExecutionHook hook = RxJavaPlugins.getInstance().getObservableExecutionHook(); - } + private final Func1, Subscription> onSubscribe; /** * Observable with Function to execute when subscribed to. @@ -131,14 +118,16 @@ public static interface OnSubscribeFunc extends Function { * specifically have a need for inheritance. * * @param onSubscribe - * {@link OnSubscribeFunc} to be executed when {@link #subscribe(Observer)} is called. + * {@link Func1} to be executed when {@link #subscribe(Observer)} is called. */ - protected Observable(OnSubscribeFunc onSubscribe) { + protected Observable(Func1, Subscription> onSubscribe) { this.onSubscribe = onSubscribe; } - private final static RxJavaObservableExecutionHook hook = RxJavaPlugins.getInstance().getObservableExecutionHook(); - + protected Observable() { + this(null); + //TODO should this be made private to prevent it? It really serves no good purpose and only confuses things. Unit tests are incorrectly using it today + } /** * An {@link Observer} must call an Observable's {@code subscribe} method in order to @@ -167,9 +156,9 @@ protected Observable(OnSubscribeFunc onSubscribe) { * @throws IllegalArgumentException * if the {@link Observer} provided as the argument to {@code subscribe()} is {@code null} */ - public Subscription subscribe(Observer observer) { + public Subscription subscribe(Observer observer) { // allow the hook to intercept and/or decorate - OnSubscribeFunc onSubscribeFunction = hook.onSubscribeStart(this, onSubscribe); + Func1, Subscription> onSubscribeFunction = hook.onSubscribeStart(this, onSubscribe); // validate and proceed if (observer == null) { throw new IllegalArgumentException("observer can not be null"); @@ -183,7 +172,7 @@ public Subscription subscribe(Observer observer) { * See https://github.com/Netflix/RxJava/issues/216 for discussion on "Guideline 6.4: Protect calls to user code from within an operator" */ if (isInternalImplementation(observer)) { - Subscription s = onSubscribeFunction.onSubscribe(observer); + Subscription s = onSubscribeFunction.call(observer); if (s == null) { // this generally shouldn't be the case on a 'trusted' onSubscribe but in case it happens // we want to gracefully handle it the same as AtomicObservableSubscription does @@ -193,7 +182,7 @@ public Subscription subscribe(Observer observer) { } } else { SafeObservableSubscription subscription = new SafeObservableSubscription(); - subscription.wrap(onSubscribeFunction.onSubscribe(new SafeObserver(subscription, observer))); + subscription.wrap(onSubscribeFunction.call(new SafeObserver(subscription, observer))); return hook.onSubscribeReturn(this, subscription); } } catch (OnErrorNotImplementedException e) { @@ -245,7 +234,7 @@ public Subscription subscribe(Observer observer) { * @throws IllegalArgumentException * if an argument to {@code subscribe()} is {@code null} */ - public Subscription subscribe(Observer observer, Scheduler scheduler) { + public Subscription subscribe(Observer observer, Scheduler scheduler) { return subscribeOn(scheduler).subscribe(observer); } @@ -254,12 +243,12 @@ public Subscription subscribe(Observer observer, Scheduler scheduler) *

* See https://github.com/Netflix/RxJava/issues/216 for discussion on "Guideline 6.4: Protect calls to user code from within an operator" */ - private Subscription protectivelyWrapAndSubscribe(Observer o) { + private Subscription protectivelyWrapAndSubscribe(Observer o) { SafeObservableSubscription subscription = new SafeObservableSubscription(); return subscription.wrap(subscribe(new SafeObserver(subscription, o))); } - public Subscription subscribe(final Action1 onNext) { + public Subscription subscribe(final Action1 onNext) { if (onNext == null) { throw new IllegalArgumentException("onNext can not be null"); } @@ -290,11 +279,11 @@ public void onNext(T args) { }); } - public Subscription subscribe(final Action1 onNext, Scheduler scheduler) { + public Subscription subscribe(final Action1 onNext, Scheduler scheduler) { return subscribeOn(scheduler).subscribe(onNext); } - public Subscription subscribe(final Action1 onNext, final Action1 onError) { + public Subscription subscribe(final Action1 onNext, final Action1 onError) { if (onNext == null) { throw new IllegalArgumentException("onNext can not be null"); } @@ -328,11 +317,11 @@ public void onNext(T args) { }); } - public Subscription subscribe(final Action1 onNext, final Action1 onError, Scheduler scheduler) { + public Subscription subscribe(final Action1 onNext, final Action1 onError, Scheduler scheduler) { return subscribeOn(scheduler).subscribe(onNext, onError); } - public Subscription subscribe(final Action1 onNext, final Action1 onError, final Action0 onComplete) { + public Subscription subscribe(final Action1 onNext, final Action1 onError, final Action0 onComplete) { if (onNext == null) { throw new IllegalArgumentException("onNext can not be null"); } @@ -369,7 +358,7 @@ public void onNext(T args) { }); } - public Subscription subscribe(final Action1 onNext, final Action1 onError, final Action0 onComplete, Scheduler scheduler) { + public Subscription subscribe(final Action1 onNext, final Action1 onError, final Action0 onComplete, Scheduler scheduler) { return subscribeOn(scheduler).subscribe(onNext, onError, onComplete); } @@ -409,10 +398,10 @@ private void handleError(Throwable e) { */ private static class NeverObservable extends Observable { public NeverObservable() { - super(new OnSubscribeFunc() { + super(new Func1, Subscription>() { @Override - public Subscription onSubscribe(Observer t1) { + public Subscription call(Observer t1) { return Subscriptions.empty(); } @@ -429,7 +418,7 @@ public Subscription onSubscribe(Observer t1) { private static class ThrowObservable extends Observable { public ThrowObservable(final Throwable exception) { - super(new OnSubscribeFunc() { + super(new Func1, Subscription>() { /** * Accepts an {@link Observer} and calls its {@link Observer#onError onError} method. @@ -439,7 +428,7 @@ public ThrowObservable(final Throwable exception) { * @return a reference to the subscription */ @Override - public Subscription onSubscribe(Observer observer) { + public Subscription call(Observer observer) { observer.onError(exception); return Subscriptions.empty(); } @@ -473,7 +462,7 @@ public Subscription onSubscribe(Observer observer) { * @return an Observable that, when an {@link Observer} subscribes to it, will execute the given * function */ - public static Observable create(OnSubscribeFunc func) { + public static Observable create(Func1, Subscription> func) { return new Observable(func); } @@ -522,7 +511,7 @@ public static Observable error(Throwable exception) { * emitted by the resulting Observable * @return an Observable that emits each item in the source {@link Iterable} sequence */ - public static Observable from(Iterable iterable) { + public static Observable from(Iterable iterable) { return create(OperationToObservableIterable.toObservableIterable(iterable)); } @@ -586,7 +575,7 @@ public static Observable range(int start, int count) { * @return an Observable whose {@link Observer}s trigger an invocation of the given Observable * factory function */ - public static Observable defer(Func0> observableFactory) { + public static Observable defer(Func0> observableFactory) { return create(OperationDefer.defer(observableFactory)); } @@ -629,7 +618,7 @@ public static Observable just(T value) { * @return an Observable that emits items that are the result of flattening the {@code source} list of Observables * @see MSDN: Observable.Merge */ - public static Observable merge(List> source) { + public static Observable merge(List> source) { return create(OperationMerge.merge(source)); } @@ -648,7 +637,7 @@ public static Observable merge(List> so * by the Observables emitted by the {@code source} Observable * @see MSDN: Observable.Merge Method */ - public static Observable merge(Observable> source) { + public static Observable merge(Observable> source) { return create(OperationMerge.merge(source)); } @@ -666,7 +655,7 @@ public static Observable merge(ObservableMSDN: Observable.Merge Method */ - public static Observable merge(Observable... source) { + public static Observable merge(Observable... source) { return create(OperationMerge.merge(source)); } @@ -682,7 +671,7 @@ public static Observable merge(Observable... source) { * the {@code source} Observables, one after the other * @see MSDN: Observable.Concat Method */ - public static Observable concat(Observable... source) { + public static Observable concat(Observable... source) { return create(OperationConcat.concat(source)); } @@ -706,7 +695,7 @@ public static Observable concat(Observable... source) { * the {@code source} list of Observables * @see MSDN: Observable.Merge Method */ - public static Observable mergeDelayError(List> source) { + public static Observable mergeDelayError(List> source) { return create(OperationMergeDelayError.mergeDelayError(source)); } @@ -730,7 +719,7 @@ public static Observable mergeDelayError(ListMSDN: Observable.Merge Method */ - public static Observable mergeDelayError(Observable> source) { + public static Observable mergeDelayError(Observable> source) { return create(OperationMergeDelayError.mergeDelayError(source)); } @@ -754,7 +743,7 @@ public static Observable mergeDelayError(ObservableMSDN: Observable.Merge Method */ - public static Observable mergeDelayError(Observable... source) { + public static Observable mergeDelayError(Observable... source) { return create(OperationMergeDelayError.mergeDelayError(source)); } @@ -784,7 +773,7 @@ public static Observable never() { * @return an Observable that emits only the items emitted by the most recently published * Observable */ - public static Observable switchDo(Observable> sequenceOfSequences) { + public static Observable switchDo(Observable> sequenceOfSequences) { // TODO should this static remain? I have left it because it is an Observable return create(OperationSwitch.switchDo(sequenceOfSequences)); } @@ -804,7 +793,7 @@ public static Observable switchDo(Observable switchDo() { // TODO can we come up with a better name than this? It should be 'switch' but that is reserved. // Perhaps 'switchOnNext'? - return create(OperationSwitch.switchDo((Observable>) this)); + return create(OperationSwitch.switchDo((Observable>) this)); } /** @@ -824,7 +813,7 @@ public Observable switchDo() { * @return an Observable that is a chronologically well-behaved version of the source * Observable, and that synchronously notifies its {@link Observer}s */ - public static Observable synchronize(Observable observable) { + public static Observable synchronize(Observable observable) { return create(OperationSynchronize.synchronize(observable)); } @@ -857,7 +846,7 @@ public Observable> timestamp() { * be emitted by the resulting Observable * @return an Observable that emits the item from the source Future */ - public static Observable from(Future future) { + public static Observable from(Future future) { return create(OperationToObservableFuture.toObservableFuture(future)); } @@ -880,7 +869,7 @@ public static Observable from(Future future) { * be emitted by the resulting Observable * @return an Observable that emits the item from the source Future */ - public static Observable from(Future future, Scheduler scheduler) { + public static Observable from(Future future, Scheduler scheduler) { return create(OperationToObservableFuture.toObservableFuture(future)).subscribeOn(scheduler); } @@ -906,7 +895,7 @@ public static Observable from(Future future, Scheduler sched * be emitted by the resulting Observable * @return an Observable that emits the item from the source {@link Future} */ - public static Observable from(Future future, long timeout, TimeUnit unit) { + public static Observable from(Future future, long timeout, TimeUnit unit) { return create(OperationToObservableFuture.toObservableFuture(future, timeout, unit)); } @@ -920,17 +909,18 @@ public static Observable from(Future future, long timeout, T * The resulting {@code Observable} returned from {@code zip} will invoke {@link Observer#onNext onNext} as many times as the number of {@code onNext} invocations * of the source Observable that emits the fewest items. * - * @param o1 + * @param w0 * one source Observable - * @param o2 + * @param w1 * another source Observable - * @param zipFunction - * a function that, when applied to an item emitted by each of the source - * Observables, results in an item that will be emitted by the resulting Observable + * @param reduceFunction + * a function that, when applied to a pair of items, each emitted by one of the two + * source Observables, results in an item that will be emitted by the resulting + * Observable * @return an Observable that emits the zipped results */ - public static Observable zip(Observable o1, Observable o2, Func2 zipFunction) { - return create(OperationZip.zip(o1, o2, zipFunction)); + public static Observable zip(Observable w0, Observable w1, Func2 reduceFunction) { + return create(OperationZip.zip(w0, w1, reduceFunction)); } /** @@ -948,7 +938,7 @@ public static Observable zip(Observable o1, Observa * @return an Observable that emits Booleans that indicate whether the corresponding items * emitted by the source Observables are equal */ - public static Observable sequenceEqual(Observable first, Observable second) { + public static Observable sequenceEqual(Observable first, Observable second) { return sequenceEqual(first, second, new Func2() { @Override public Boolean call(T first, T second) { @@ -975,7 +965,7 @@ public Boolean call(T first, T second) { * @return an Observable that emits Booleans that indicate whether the corresponding items * emitted by the source Observables are equal */ - public static Observable sequenceEqual(Observable first, Observable second, Func2 equality) { + public static Observable sequenceEqual(Observable first, Observable second, Func2 equality) { return zip(first, second, equality); } @@ -992,19 +982,19 @@ public static Observable sequenceEqual(Observable firs * The resulting {@code Observable} returned from {@code zip} will invoke {@link Observer#onNext onNext} as many times as the number of {@code onNext} invocations * of the source Observable that emits the fewest items. * - * @param o1 + * @param w0 * one source Observable - * @param o2 - * a second source Observable - * @param o3 + * @param w1 + * another source Observable + * @param w2 * a third source Observable - * @param zipFunction + * @param function * a function that, when applied to an item emitted by each of the source * Observables, results in an item that will be emitted by the resulting Observable * @return an Observable that emits the zipped results */ - public static Observable zip(Observable o1, Observable o2, Observable o3, Func3 zipFunction) { - return create(OperationZip.zip(o1, o2, o3, zipFunction)); + public static Observable zip(Observable w0, Observable w1, Observable w2, Func3 function) { + return create(OperationZip.zip(w0, w1, w2, function)); } /** @@ -1021,210 +1011,21 @@ public static Observable zip(Observable o1, Obs * The resulting {@code Observable} returned from {@code zip} will invoke {@link Observer#onNext onNext} as many times as the number of {@code onNext} invocations * of the source Observable that emits the fewest items. * - * @param o1 - * one source Observable - * @param o2 - * a second source Observable - * @param o3 - * a third source Observable - * @param o4 - * a fourth source Observable - * @param zipFunction - * a function that, when applied to an item emitted by each of the source - * Observables, results in an item that will be emitted by the resulting Observable - * @return an Observable that emits the zipped results - */ - public static Observable zip(Observable o1, Observable o2, Observable o3, Observable o4, Func4 zipFunction) { - return create(OperationZip.zip(o1, o2, o3, o4, zipFunction)); - } - - /** - * Returns an Observable that emits the results of a function of your choosing applied to - * combinations of four items emitted, in sequence, by four other Observables. - *

- * - *

{@code zip} applies this function in strict sequence, so the first item emitted by the - * new Observable will be the result of the function applied to the first item emitted by {@code w0}, the first item emitted by {@code w1}, the first item emitted by {@code w2}, and the first item - * emitted by {@code w3}; the second item emitted by - * the new Observable will be the result of the function applied to the second item emitted by - * each of those Observables; and so forth. - *

- * The resulting {@code Observable} returned from {@code zip} will invoke {@link Observer#onNext onNext} as many times as the number of {@code onNext} invocations - * of the source Observable that emits the fewest items. - * - * @param o1 - * one source Observable - * @param o2 - * a second source Observable - * @param o3 - * a third source Observable - * @param o4 - * a fourth source Observable - * @param o5 - * a fifth source Observable - * @param zipFunction - * a function that, when applied to an item emitted by each of the source - * Observables, results in an item that will be emitted by the resulting Observable - * @return an Observable that emits the zipped results - */ - public static Observable zip(Observable o1, Observable o2, Observable o3, Observable o4, Observable o5, Func5 zipFunction) { - return create(OperationZip.zip(o1, o2, o3, o4, o5, zipFunction)); - } - - /** - * Returns an Observable that emits the results of a function of your choosing applied to - * combinations of four items emitted, in sequence, by four other Observables. - *

- * - *

{@code zip} applies this function in strict sequence, so the first item emitted by the - * new Observable will be the result of the function applied to the first item emitted by {@code w0}, the first item emitted by {@code w1}, the first item emitted by {@code w2}, and the first item - * emitted by {@code w3}; the second item emitted by - * the new Observable will be the result of the function applied to the second item emitted by - * each of those Observables; and so forth. - *

- * The resulting {@code Observable} returned from {@code zip} will invoke {@link Observer#onNext onNext} as many times as the number of {@code onNext} invocations - * of the source Observable that emits the fewest items. - * - * @param o1 - * one source Observable - * @param o2 - * a second source Observable - * @param o3 - * a third source Observable - * @param o4 - * a fourth source Observable - * @param o5 - * a fifth source Observable - * @param o6 - * a sixth source Observable - * @param zipFunction - * a function that, when applied to an item emitted by each of the source - * Observables, results in an item that will be emitted by the resulting Observable - * @return an Observable that emits the zipped results - */ - public static Observable zip(Observable o1, Observable o2, Observable o3, Observable o4, Observable o5, Observable o6, - Func6 zipFunction) { - return create(OperationZip.zip(o1, o2, o3, o4, o5, o6, zipFunction)); - } - - /** - * Returns an Observable that emits the results of a function of your choosing applied to - * combinations of four items emitted, in sequence, by four other Observables. - *

- * - *

{@code zip} applies this function in strict sequence, so the first item emitted by the - * new Observable will be the result of the function applied to the first item emitted by {@code w0}, the first item emitted by {@code w1}, the first item emitted by {@code w2}, and the first item - * emitted by {@code w3}; the second item emitted by - * the new Observable will be the result of the function applied to the second item emitted by - * each of those Observables; and so forth. - *

- * The resulting {@code Observable} returned from {@code zip} will invoke {@link Observer#onNext onNext} as many times as the number of {@code onNext} invocations - * of the source Observable that emits the fewest items. - * - * @param o1 + * @param w0 * one source Observable - * @param o2 - * a second source Observable - * @param o3 - * a third source Observable - * @param o4 - * a fourth source Observable - * @param o5 - * a fifth source Observable - * @param o6 - * a sixth source Observable - * @param o7 - * a seventh source Observable - * @param zipFunction - * a function that, when applied to an item emitted by each of the source - * Observables, results in an item that will be emitted by the resulting Observable - * @return an Observable that emits the zipped results - */ - public static Observable zip(Observable o1, Observable o2, Observable o3, Observable o4, Observable o5, Observable o6, Observable o7, - Func7 zipFunction) { - return create(OperationZip.zip(o1, o2, o3, o4, o5, o6, o7, zipFunction)); - } - - /** - * Returns an Observable that emits the results of a function of your choosing applied to - * combinations of four items emitted, in sequence, by four other Observables. - *

- * - *

{@code zip} applies this function in strict sequence, so the first item emitted by the - * new Observable will be the result of the function applied to the first item emitted by {@code w0}, the first item emitted by {@code w1}, the first item emitted by {@code w2}, and the first item - * emitted by {@code w3}; the second item emitted by - * the new Observable will be the result of the function applied to the second item emitted by - * each of those Observables; and so forth. - *

- * The resulting {@code Observable} returned from {@code zip} will invoke {@link Observer#onNext onNext} as many times as the number of {@code onNext} invocations - * of the source Observable that emits the fewest items. - * - * @param o1 - * one source Observable - * @param o2 - * a second source Observable - * @param o3 - * a third source Observable - * @param o4 - * a fourth source Observable - * @param o5 - * a fifth source Observable - * @param o6 - * a sixth source Observable - * @param o7 - * a seventh source Observable - * @param o8 - * an eighth source Observable - * @param zipFunction - * a function that, when applied to an item emitted by each of the source - * Observables, results in an item that will be emitted by the resulting Observable - * @return an Observable that emits the zipped results - */ - public static Observable zip(Observable o1, Observable o2, Observable o3, Observable o4, Observable o5, Observable o6, Observable o7, Observable o8, - Func8 zipFunction) { - return create(OperationZip.zip(o1, o2, o3, o4, o5, o6, o7, o8, zipFunction)); - } - - /** - * Returns an Observable that emits the results of a function of your choosing applied to - * combinations of four items emitted, in sequence, by four other Observables. - *

- * - *

{@code zip} applies this function in strict sequence, so the first item emitted by the - * new Observable will be the result of the function applied to the first item emitted by {@code w0}, the first item emitted by {@code w1}, the first item emitted by {@code w2}, and the first item - * emitted by {@code w3}; the second item emitted by - * the new Observable will be the result of the function applied to the second item emitted by - * each of those Observables; and so forth. - *

- * The resulting {@code Observable} returned from {@code zip} will invoke {@link Observer#onNext onNext} as many times as the number of {@code onNext} invocations - * of the source Observable that emits the fewest items. - * - * @param o1 - * one source Observable - * @param o2 - * a second source Observable - * @param o3 + * @param w1 + * another source Observable + * @param w2 * a third source Observable - * @param o4 + * @param w3 * a fourth source Observable - * @param o5 - * a fifth source Observable - * @param o6 - * a sixth source Observable - * @param o7 - * a seventh source Observable - * @param o8 - * an eighth source Observable - * @param o9 - * a ninth source Observable - * @param zipFunction + * @param reduceFunction * a function that, when applied to an item emitted by each of the source * Observables, results in an item that will be emitted by the resulting Observable * @return an Observable that emits the zipped results */ - public static Observable zip(Observable o1, Observable o2, Observable o3, Observable o4, Observable o5, Observable o6, Observable o7, Observable o8, - Observable o9, Func9 zipFunction) { - return create(OperationZip.zip(o1, o2, o3, o4, o5, o6, o7, o8, o9, zipFunction)); + public static Observable zip(Observable w0, Observable w1, Observable w2, Observable w3, Func4 reduceFunction) { + return create(OperationZip.zip(w0, w1, w2, w3, reduceFunction)); } /** @@ -1233,89 +1034,48 @@ public static Observable zip(Observab *

* * - * @param o1 + * @param w0 * The first source observable. - * @param o2 + * @param w1 * The second source observable. * @param combineFunction * The aggregation function used to combine the source observable values. * @return An Observable that combines the source Observables with the given combine function */ - public static Observable combineLatest(Observable o1, Observable o2, Func2 combineFunction) { - return create(OperationCombineLatest.combineLatest(o1, o2, combineFunction)); - } - - /** - * @see #combineLatest(Observable, Observable, Func2) - */ - public static Observable combineLatest(Observable o1, Observable o2, Observable o3, Func3 combineFunction) { - return create(OperationCombineLatest.combineLatest(o1, o2, o3, combineFunction)); - } - - /** - * @see #combineLatest(Observable, Observable, Func2) - */ - public static Observable combineLatest(Observable o1, Observable o2, Observable o3, Observable o4, - Func4 combineFunction) { - return create(OperationCombineLatest.combineLatest(o1, o2, o3, o4, combineFunction)); + public static Observable combineLatest(Observable w0, Observable w1, Func2 combineFunction) { + return create(OperationCombineLatest.combineLatest(w0, w1, combineFunction)); } /** * @see #combineLatest(Observable, Observable, Func2) */ - public static Observable combineLatest(Observable o1, Observable o2, Observable o3, Observable o4, Observable o5, - Func5 combineFunction) { - return create(OperationCombineLatest.combineLatest(o1, o2, o3, o4, o5, combineFunction)); + public static Observable combineLatest(Observable w0, Observable w1, Observable w2, Func3 combineFunction) { + return create(OperationCombineLatest.combineLatest(w0, w1, w2, combineFunction)); } /** * @see #combineLatest(Observable, Observable, Func2) */ - public static Observable combineLatest(Observable o1, Observable o2, Observable o3, Observable o4, Observable o5, Observable o6, - Func6 combineFunction) { - return create(OperationCombineLatest.combineLatest(o1, o2, o3, o4, o5, o6, combineFunction)); - } - - /** - * @see #combineLatest(Observable, Observable, Func2) - */ - public static Observable combineLatest(Observable o1, Observable o2, Observable o3, Observable o4, Observable o5, Observable o6, Observable o7, - Func7 combineFunction) { - return create(OperationCombineLatest.combineLatest(o1, o2, o3, o4, o5, o6, o7, combineFunction)); - } - - /** - * @see #combineLatest(Observable, Observable, Func2) - */ - public static Observable combineLatest(Observable o1, Observable o2, Observable o3, Observable o4, Observable o5, Observable o6, Observable o7, Observable o8, - Func8 combineFunction) { - return create(OperationCombineLatest.combineLatest(o1, o2, o3, o4, o5, o6, o7, o8, combineFunction)); - } - - /** - * @see #combineLatest(Observable, Observable, Func2) - */ - public static Observable combineLatest(Observable o1, Observable o2, Observable o3, Observable o4, Observable o5, Observable o6, Observable o7, Observable o8, Observable o9, - Func9 combineFunction) { - return create(OperationCombineLatest.combineLatest(o1, o2, o3, o4, o5, o6, o7, o8, o9, combineFunction)); + public static Observable combineLatest(Observable w0, Observable w1, Observable w2, Observable w3, Func4 combineFunction) { + return create(OperationCombineLatest.combineLatest(w0, w1, w2, w3, combineFunction)); } /** * 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 {@link Func0} produces a {@link BufferClosing} object. The * {@link Func0} will then + * emitted and replaced with a new buffer when the Observable produced by the specified {@link Func0} produces a {@link rx.util.Closing} object. The * {@link Func0} 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 BufferClosing} object, the associated buffer + * When this {@link Observable} produces a {@link rx.util.Closing} 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 BufferClosing} object. + * when the current {@link Observable} created with the {@link Func0} argument produces a {@link rx.util.Closing} object. */ - public Observable> buffer(Func0> bufferClosingSelector) { + public Observable> buffer(Func0> bufferClosingSelector) { return create(OperationBuffer.buffer(this, bufferClosingSelector)); } @@ -1323,21 +1083,21 @@ public Observable> buffer(Func0This Observable produces buffers. Buffers are created when the specified "bufferOpenings" - * Observable produces a {@link BufferOpening} object. Additionally the {@link Func0} argument - * is used to create an Observable which produces {@link BufferClosing} objects. When this + * Observable produces a {@link rx.util.Opening} object. Additionally the {@link Func0} argument + * is used to create an Observable which produces {@link rx.util.Closing} objects. When this * Observable produces such an object, the associated buffer is emitted. * * @param bufferOpenings - * The {@link Observable} which, when it produces a {@link BufferOpening} object, will cause + * The {@link Observable} which, when it produces a {@link rx.util.Opening} object, will cause * another buffer to be created. * @param bufferClosingSelector * The {@link Func0} which is used to produce an {@link Observable} for every buffer created. - * When this {@link Observable} produces a {@link BufferClosing} object, the associated buffer + * When this {@link Observable} produces a {@link rx.util.Closing} object, the associated buffer * is emitted. * @return * An {@link Observable} which produces buffers which are created and emitted when the specified {@link Observable}s publish certain objects. */ - public Observable> buffer(Observable bufferOpenings, Func1> bufferClosingSelector) { + public Observable> buffer(Observable bufferOpenings, Func1> bufferClosingSelector) { return create(OperationBuffer.buffer(this, bufferOpenings, bufferClosingSelector)); } @@ -1503,7 +1263,222 @@ public Observable> buffer(long timespan, long timeshift, TimeUnit unit) public Observable> buffer(long timespan, long timeshift, TimeUnit unit, Scheduler scheduler) { return create(OperationBuffer.buffer(this, timespan, timeshift, unit, scheduler)); } + + /** + * 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 {@link Func0} produces a {@link rx.util.Closing} object. The + * {@link Func0} will then be used to create a new Observable to listen for the end of the next window. + * + * @param source + * The source {@link Observable} which produces values. + * @param closingSelector + * The {@link Func0} which is used to produce an {@link Observable} for every window created. + * When this {@link Observable} produces a {@link rx.util.Closing} object, the associated window + * is emitted and replaced with a new one. + * @return + * An {@link Observable} which produces connected non-overlapping windows, which are emitted + * when the current {@link Observable} created with the {@link Func0} argument produces a + * {@link rx.util.Closing} object. + */ + public Observable> window(Observable source, Func0> closingSelector) { + return create(OperationWindow.window(source, closingSelector)); + } + + /** + * Creates an Observable which produces windows of collected values. This Observable produces windows. + * Chunks are created when the specified "windowOpenings" Observable produces a {@link rx.util.Opening} object. + * Additionally the {@link Func0} argument is used to create an Observable which produces {@link rx.util.Closing} + * objects. When this Observable produces such an object, the associated window is emitted. + * + * @param source + * The source {@link Observable} which produces values. + * @param windowOpenings + * The {@link Observable} which when it produces a {@link rx.util.Opening} object, will cause + * another window to be created. + * @param closingSelector + * The {@link Func0} which is used to produce an {@link Observable} for every window created. + * When this {@link Observable} produces a {@link rx.util.Closing} object, the associated window + * is emitted. + * @return + * An {@link Observable} which produces windows which are created and emitted when the specified + * {@link Observable}s publish certain objects. + */ + public Observable> window(Observable source, Observable windowOpenings, Func1> closingSelector) { + return create(OperationWindow.window(source, windowOpenings, closingSelector)); + } + /** + * Creates an Observable which produces windows of collected values. This Observable produces connected + * non-overlapping windows, each containing "count" elements. When the source Observable completes or + * encounters an error, the current window is emitted, and the event is propagated. + * + * @param source + * The source {@link Observable} which produces values. + * @param count + * The maximum size of each window before it should be emitted. + * @return + * An {@link Observable} which produces connected non-overlapping windows containing at most + * "count" produced values. + */ + public Observable> window(Observable source, int count) { + return create(OperationWindow.window(source, count)); + } + + /** + * Creates an Observable which produces windows of collected values. This Observable produces windows every + * "skip" values, each containing "count" elements. When the source Observable completes or encounters an error, + * the current window is emitted and the event is propagated. + * + * @param source + * The source {@link Observable} which produces values. + * @param count + * The maximum size of each window before it should be emitted. + * @param skip + * How many produced values need to be skipped before starting a new window. Note that when "skip" and + * "count" are equals that this is the same operation as {@link Observable#window(Observable, int)}. + * @return + * An {@link Observable} which produces windows every "skipped" values containing at most + * "count" produced values. + */ + public Observable> window(Observable source, int count, int skip) { + return create(OperationWindow.window(source, count, skip)); + } + + /** + * Creates an Observable which produces windows of collected values. This Observable produces connected + * non-overlapping windows, each of a fixed duration specified by the "timespan" argument. When the source + * Observable completes or encounters an error, the current window is emitted and the event is propagated. + * + * @param source + * The source {@link Observable} which produces values. + * @param timespan + * The period of time each window is collecting values before it should be emitted, and + * replaced with a new window. + * @param unit + * The unit of time which applies to the "timespan" argument. + * @return + * An {@link Observable} which produces connected non-overlapping windows with a fixed duration. + */ + public Observable> window(Observable source, long timespan, TimeUnit unit) { + return create(OperationWindow.window(source, timespan, unit)); + } + + /** + * Creates an Observable which produces windows of collected values. This Observable produces connected + * non-overlapping windows, each of a fixed duration specified by the "timespan" argument. When the source + * Observable completes or encounters an error, the current window is emitted and the event is propagated. + * + * @param source + * The source {@link Observable} which produces values. + * @param timespan + * The period of time each window is collecting values before it should be emitted, and + * replaced with a new window. + * @param unit + * The unit of time which applies to the "timespan" argument. + * @param scheduler + * The {@link Scheduler} to use when determining the end and start of a window. + * @return + * An {@link Observable} which produces connected non-overlapping windows with a fixed duration. + */ + public Observable> window(Observable source, long timespan, TimeUnit unit, Scheduler scheduler) { + return create(OperationWindow.window(source, timespan, unit, scheduler)); + } + + /** + * Creates an Observable which produces windows of collected values. This Observable produces connected + * non-overlapping windows, each of a fixed duration specified by the "timespan" argument or a maximum size + * specified by the "count" argument (which ever is reached first). When the source Observable completes + * or encounters an error, the current window is emitted and the event is propagated. + * + * @param source + * The source {@link Observable} which produces values. + * @param timespan + * The period of time each window is collecting values before it should be emitted, and + * replaced with a new window. + * @param unit + * The unit of time which applies to the "timespan" argument. + * @param count + * The maximum size of each window before it should be emitted. + * @return + * An {@link Observable} which produces connected non-overlapping windows which are emitted after + * a fixed duration or when the window has reached maximum capacity (which ever occurs first). + */ + public Observable> window(Observable source, long timespan, TimeUnit unit, int count) { + return create(OperationWindow.window(source, timespan, unit, count)); + } + + /** + * Creates an Observable which produces windows of collected values. This Observable produces connected + * non-overlapping windows, each of a fixed duration specified by the "timespan" argument or a maximum size + * specified by the "count" argument (which ever is reached first). When the source Observable completes + * or encounters an error, the current window is emitted and the event is propagated. + * + * @param source + * The source {@link Observable} which produces values. + * @param timespan + * The period of time each window is collecting values before it should be emitted, and + * replaced with a new window. + * @param unit + * The unit of time which applies to the "timespan" argument. + * @param count + * The maximum size of each window before it should be emitted. + * @param scheduler + * The {@link Scheduler} to use when determining the end and start of a window. + * @return + * An {@link Observable} which produces connected non-overlapping windows which are emitted after + * a fixed duration or when the window has reached maximum capacity (which ever occurs first). + */ + public Observable> window(Observable source, long timespan, TimeUnit unit, int count, Scheduler scheduler) { + return create(OperationWindow.window(source, timespan, unit, count, scheduler)); + } + + /** + * Creates an Observable which produces windows of collected values. This Observable starts a new window + * periodically, which is determined by the "timeshift" argument. Each window is emitted after a fixed timespan + * specified by the "timespan" argument. When the source Observable completes or encounters an error, the + * current window is emitted and the event is propagated. + * + * @param source + * The source {@link Observable} which produces values. + * @param timespan + * The period of time each window is collecting values before it should be emitted. + * @param timeshift + * The period of time after which a new window will be created. + * @param unit + * The unit of time which applies to the "timespan" and "timeshift" argument. + * @return + * An {@link Observable} which produces new windows periodically, and these are emitted after + * a fixed timespan has elapsed. + */ + public Observable> window(Observable source, long timespan, long timeshift, TimeUnit unit) { + return create(OperationWindow.window(source, timespan, timeshift, unit)); + } + + /** + * Creates an Observable which produces windows of collected values. This Observable starts a new window + * periodically, which is determined by the "timeshift" argument. Each window is emitted after a fixed timespan + * specified by the "timespan" argument. When the source Observable completes or encounters an error, the + * current window is emitted and the event is propagated. + * + * @param source + * The source {@link Observable} which produces values. + * @param timespan + * The period of time each window is collecting values before it should be emitted. + * @param timeshift + * The period of time after which a new window will be created. + * @param unit + * The unit of time which applies to the "timespan" and "timeshift" argument. + * @param scheduler + * The {@link Scheduler} to use when determining the end and start of a window. + * @return + * An {@link Observable} which produces new windows periodically, and these are emitted after + * a fixed timespan has elapsed. + */ + public Observable> window(Observable source, long timespan, long timeshift, TimeUnit unit, Scheduler scheduler) { + return create(OperationWindow.window(source, timespan, timeshift, unit, scheduler)); + } + /** * Returns an Observable that emits the results of a function of your choosing applied to * combinations of four items emitted, in sequence, by four other Observables. @@ -1519,16 +1494,16 @@ public Observable> buffer(long timespan, long timeshift, TimeUnit unit, * * @param ws * An Observable of source Observables - * @param zipFunction + * @param reduceFunction * a function that, when applied to an item emitted by each of the source * Observables, results in an item that will be emitted by the resulting Observable * @return an Observable that emits the zipped results */ - public static Observable zip(Observable> ws, final FuncN zipFunction) { - return ws.toList().mapMany(new Func1>, Observable>() { + public static Observable zip(Observable> ws, final FuncN reduceFunction) { + return ws.toList().mapMany(new Func1>, Observable>() { @Override - public Observable call(List> wsList) { - return create(OperationZip.zip(wsList, zipFunction)); + public Observable call(List> wsList) { + return create(OperationZip.zip(wsList, reduceFunction)); } }); } @@ -1548,13 +1523,13 @@ public Observable call(List> wsList) { * * @param ws * A collection of source Observables - * @param zipFunction + * @param reduceFunction * a function that, when applied to an item emitted by each of the source * Observables, results in an item that will be emitted by the resulting Observable * @return an Observable that emits the zipped results */ - public static Observable zip(Collection> ws, FuncN zipFunction) { - return create(OperationZip.zip(ws, zipFunction)); + public static Observable zip(Collection> ws, FuncN reduceFunction) { + return create(OperationZip.zip(ws, reduceFunction)); } /** @@ -1566,7 +1541,7 @@ public static Observable zip(Collection> ws, Func * @return an Observable that emits only those items in the original Observable that the filter * evaluates as {@code true} */ - public Observable filter(Func1 predicate) { + public Observable filter(Func1 predicate) { return create(OperationFilter.filter(this, predicate)); } @@ -1601,7 +1576,7 @@ public Observable finallyDo(Action0 action) { * obtained from this transformation. * @see #mapMany(Func1) */ - public Observable flatMap(Func1> func) { + public Observable flatMap(Func1> func) { return mapMany(func); } @@ -1615,7 +1590,7 @@ public Observable flatMap(Func1 where(Func1 predicate) { + public Observable where(Func1 predicate) { return filter(predicate); } @@ -1630,7 +1605,7 @@ public Observable where(Func1 predicate) { * @return an Observable that emits the items from the source Observable, transformed by the * given function */ - public Observable map(Func1 func) { + public Observable map(Func1 func) { return create(OperationMap.map(this, func)); } @@ -1651,7 +1626,7 @@ public Observable map(Func1 func) { * obtained from this transformation. * @see #flatMap(Func1) */ - public Observable mapMany(Func1> func) { + public Observable mapMany(Func1> func) { return create(OperationMap.mapMany(this, func)); } @@ -1710,7 +1685,7 @@ public Observable observeOn(Scheduler scheduler) { */ @SuppressWarnings("unchecked") public Observable dematerialize() { - return create(OperationDematerialize.dematerialize((Observable>) this)); + return create(OperationDematerialize.dematerialize((Observable>) this)); } /** @@ -1737,7 +1712,7 @@ public Observable dematerialize() { * encounters an error * @return the original Observable, with appropriately modified behavior */ - public Observable onErrorResumeNext(final Func1> resumeFunction) { + public Observable onErrorResumeNext(final Func1> resumeFunction) { return create(OperationOnErrorResumeNextViaFunction.onErrorResumeNextViaFunction(this, resumeFunction)); } @@ -1765,7 +1740,7 @@ public Observable onErrorResumeNext(final Func1 onErrorResumeNext(final Observable resumeSequence) { + public Observable onErrorResumeNext(final Observable resumeSequence) { return create(OperationOnErrorResumeNextViaObservable.onErrorResumeNextViaObservable(this, resumeSequence)); } @@ -1795,7 +1770,7 @@ public Observable onErrorResumeNext(final Observable resumeSeque * encounters an error * @return the original Observable, with appropriately modified behavior */ - public Observable onExceptionResumeNext(final Observable resumeSequence) { + public Observable onExceptionResumeNext(final Observable resumeSequence) { return create(OperationOnExceptionResumeNextViaObservable.onExceptionResumeNextViaObservable(this, resumeSequence)); } @@ -1822,7 +1797,7 @@ public Observable onExceptionResumeNext(final Observable resumeS * Observable encounters an error * @return the original Observable with appropriately modified behavior */ - public Observable onErrorReturn(Func1 resumeFunction) { + public Observable onErrorReturn(Func1 resumeFunction) { return create(OperationOnErrorReturn.onErrorReturn(this, resumeFunction)); } @@ -1847,7 +1822,7 @@ public Observable onErrorReturn(Func1 resumeFunction) * @see MSDN: Observable.Aggregate * @see Wikipedia: Fold (higher-order function) */ - public Observable reduce(Func2 accumulator) { + public Observable reduce(Func2 accumulator) { return create(OperationScan.scan(this, accumulator)).takeLast(1); } @@ -1904,7 +1879,7 @@ public ConnectableObservable publish() { * * @see #reduce(Func2) */ - public Observable aggregate(Func2 accumulator) { + public Observable aggregate(Func2 accumulator) { return reduce(accumulator); } @@ -1931,7 +1906,7 @@ public Observable aggregate(Func2 accumula * @see MSDN: Observable.Aggregate * @see Wikipedia: Fold (higher-order function) */ - public Observable reduce(R initialValue, Func2 accumulator) { + public Observable reduce(R initialValue, Func2 accumulator) { return create(OperationScan.scan(this, initialValue, accumulator)).takeLast(1); } @@ -1942,7 +1917,7 @@ public Observable reduce(R initialValue, Func2 Observable aggregate(R initialValue, Func2 accumulator) { + public Observable aggregate(R initialValue, Func2 accumulator) { return reduce(initialValue, accumulator); } @@ -1965,7 +1940,7 @@ public Observable aggregate(R initialValue, Func2MSDN: Observable.Scan */ - public Observable scan(Func2 accumulator) { + public Observable scan(Func2 accumulator) { return create(OperationScan.scan(this, accumulator)); } @@ -2026,7 +2001,7 @@ public Observable sample(long period, TimeUnit unit, Scheduler scheduler) { * @return an Observable that emits the results of each call to the accumulator function * @see MSDN: Observable.Scan */ - public Observable scan(R initialValue, Func2 accumulator) { + public Observable scan(R initialValue, Func2 accumulator) { return create(OperationScan.scan(this, initialValue, accumulator)); } @@ -2041,7 +2016,7 @@ public Observable scan(R initialValue, Func2true if all items emitted by the source * Observable satisfy the predicate; otherwise, false */ - public Observable all(Func1 predicate) { + public Observable all(Func1 predicate) { return create(OperationAll.all(this, predicate)); } @@ -2094,7 +2069,7 @@ public Observable take(final int num) { * @return an Observable that emits the items from the source Observable so long as each item * satisfies the condition defined by predicate */ - public Observable takeWhile(final Func1 predicate) { + public Observable takeWhile(final Func1 predicate) { return create(OperationTakeWhile.takeWhile(this, predicate)); } @@ -2111,7 +2086,7 @@ public Observable takeWhile(final Func1 predicate) { * @return an Observable that emits items from the source Observable so long as the predicate * continues to return true for each item, then completes */ - public Observable takeWhileWithIndex(final Func2 predicate) { + public Observable takeWhileWithIndex(final Func2 predicate) { return create(OperationTakeWhile.takeWhileWithIndex(this, predicate)); } @@ -2145,7 +2120,7 @@ public Observable takeLast(final int count) { * @return an Observable that emits the items of the source Observable until such time as * other emits its first item */ - public Observable takeUntil(Observable other) { + public Observable takeUntil(Observable other) { return OperationTakeUntil.takeUntil(this, other); } @@ -2197,7 +2172,7 @@ public Observable> toSortedList() { * an Integer that indicates their sort order * @return an Observable that emits the items from the source Observable in sorted order */ - public Observable> toSortedList(Func2 sortFunction) { + public Observable> toSortedList(Func2 sortFunction) { return create(OperationToObservableSortedList.toSortedList(this, sortFunction)); } @@ -2233,7 +2208,7 @@ public Observable startWith(T... values) { * unique key value and emits items representing items from the source Observable that * share that key value */ - public Observable> groupBy(final Func1 keySelector, final Func1 elementSelector) { + public Observable> groupBy(final Func1 keySelector, final Func1 elementSelector) { return create(OperationGroupBy.groupBy(this, keySelector, elementSelector)); } @@ -2251,7 +2226,7 @@ public Observable> groupBy(final Func1 Observable> groupBy(final Func1 keySelector) { + public Observable> groupBy(final Func1 keySelector) { return create(OperationGroupBy.groupBy(this, keySelector)); } @@ -2272,7 +2247,7 @@ public BlockingObservable toBlockingObservable() { * * NOTE: If strong reasons for not depending on package names comes up then the implementation of this method can change to looking for a marker interface. * - * @param f + * @param o * @return {@code true} if the given function is an internal implementation, and {@code false} otherwise. */ private boolean isInternalImplementation(Object o) { diff --git a/rxjava-core/src/main/java/rx/operators/ChunkedOperation.java b/rxjava-core/src/main/java/rx/operators/ChunkedOperation.java new file mode 100644 index 0000000000..c7bb5cddbb --- /dev/null +++ b/rxjava-core/src/main/java/rx/operators/ChunkedOperation.java @@ -0,0 +1,587 @@ +/** + * 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 rx.Observable; +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; +import rx.util.functions.Func1; + +import java.util.ArrayList; +import java.util.List; +import java.util.Queue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * The base class for operations that break observables into "chunks". Currently buffers and windows. + */ +public class ChunkedOperation { + /** + * This interface defines a way which specifies when to create a new internal {@link rx.operators.ChunkedOperation.Chunk} object. + * + */ + protected interface ChunkCreator { + /** + * Signifies a onNext event. + */ + void onValuePushed(); + + /** + * Signifies a onCompleted or onError event. Should be used to clean up open + * subscriptions and other still running background tasks. + */ + void stop(); + } + + /** + * This class represents a single chunk: A sequence of recorded values. + * + * @param The type of objects which this {@link Chunk} can hold. + * The type of object being tracked by the {@link Chunk} + */ + protected abstract static class Chunk { + protected final List contents = new ArrayList(); + + /** + * Appends a specified value to the {@link Chunk}. + * + * @param value + * The value to append to the {@link Chunk}. + */ + public void pushValue(T value) { + contents.add(value); + } + + /** + * @return + * The mutable underlying {@link C} which contains all the + * recorded values in this {@link Chunk} object. + */ + abstract public C getContents(); + + /** + * @return + * The size of the underlying {@link List} which contains all the + * recorded values in this {@link Chunk} object. + */ + public int size() { + return contents.size(); + } + } + + /** + * This class is an extension on the {@link rx.operators.ChunkedOperation.Chunks} class which only supports one + * active (not yet emitted) internal {@link rx.operators.ChunkedOperation.Chunks} object. + * + * @param The type of object all internal {@link rx.operators.ChunkedOperation.Chunks} objects record. + * The type of object being tracked by the {@link Chunk} + */ + protected static class NonOverlappingChunks extends Chunks { + + private final Object lock = new Object(); + + public NonOverlappingChunks(Observer observer, Func0 chunkMaker) { + super(observer, chunkMaker); + } + + public Chunk emitAndReplaceChunk() { + synchronized (lock) { + emitChunk(getChunk()); + return createChunk(); + } + } + + @Override + public void pushValue(T value) { + synchronized (lock) { + super.pushValue(value); + } + } + } + + /** + * This class is an extension on the {@link rx.operators.ChunkedOperation.Chunks} class which actually has no additional + * behavior than its super class. Classes extending this class, are expected to support + * two or more active (not yet emitted) internal {@link rx.operators.ChunkedOperation.Chunks} objects. + * + * @param The type of object all internal {@link rx.operators.ChunkedOperation.Chunks} objects record. + * The type of object being tracked by the {@link rx.operators.ChunkedOperation.Chunk} + */ + protected static class OverlappingChunks extends Chunks { + public OverlappingChunks(Observer observer, Func0 chunkMaker) { + super(observer, chunkMaker); + } + } + + /** + * This class is an extension on the {@link rx.operators.ChunkedOperation.Chunks} class. Every internal chunk has + * a has a maximum time to live and a maximum internal capacity. When the chunk has + * reached the end of its life, or reached its maximum internal capacity it is + * automatically emitted. + * + * @param 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 TimeAndSizeBasedChunks extends Chunks { + + private final ConcurrentMap, Subscription> subscriptions = new ConcurrentHashMap, Subscription>(); + + private final Scheduler scheduler; + private final long maxTime; + private final TimeUnit unit; + private final int maxSize; + + public TimeAndSizeBasedChunks(Observer observer, Func0 chunkMaker, int maxSize, long maxTime, TimeUnit unit, Scheduler scheduler) { + super(observer, chunkMaker); + this.maxSize = maxSize; + this.maxTime = maxTime; + this.unit = unit; + this.scheduler = scheduler; + } + + @Override + public Chunk createChunk() { + final Chunk chunk = super.createChunk(); + subscriptions.put(chunk, scheduler.schedule(new Action0() { + @Override + public void call() { + emitChunk(chunk); + } + }, maxTime, unit)); + return chunk; + } + + @Override + public void emitChunk(Chunk chunk) { + Subscription subscription = subscriptions.remove(chunk); + if (subscription == null) { + // Chunk was already emitted. + return; + } + + subscription.unsubscribe(); + super.emitChunk(chunk); + createChunk(); + } + + @Override + public void pushValue(T value) { + super.pushValue(value); + + Chunk chunk; + while ((chunk = getChunk()) != null) { + if (chunk.size() >= maxSize) { + emitChunk(chunk); + } else { + // Chunk is not at full capacity yet, and neither will remaining chunks be so we can terminate. + break; + } + } + } + } + + /** + * This class is an extension on the {@link rx.operators.ChunkedOperation.Chunks} class. Every internal chunk has + * a has a maximum time to live. When the chunk has reached the end of its life it is + * automatically emitted. + * + * @param 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 TimeBasedChunks extends OverlappingChunks { + + private final ConcurrentMap, Subscription> subscriptions = new ConcurrentHashMap, Subscription>(); + + private final Scheduler scheduler; + private final long time; + private final TimeUnit unit; + + public TimeBasedChunks(Observer observer, Func0 chunkMaker, long time, TimeUnit unit, Scheduler scheduler) { + super(observer, chunkMaker); + this.time = time; + this.unit = unit; + this.scheduler = scheduler; + } + + @Override + public Chunk createChunk() { + final Chunk chunk = super.createChunk(); + subscriptions.put(chunk, scheduler.schedule(new Action0() { + @Override + public void call() { + emitChunk(chunk); + } + }, time, unit)); + return chunk; + } + + @Override + public void emitChunk(Chunk chunk) { + subscriptions.remove(chunk); + super.emitChunk(chunk); + } + } + + /** + * This class is an extension on the {@link rx.operators.ChunkedOperation.Chunks} class. Every internal chunk has + * a fixed maximum capacity. When the chunk has reached its maximum capacity it is + * automatically emitted. + * + * @param 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 SizeBasedChunks extends Chunks { + + private final int size; + + public SizeBasedChunks(Observer observer, Func0 chunkMaker, int size) { + super(observer, chunkMaker); + this.size = size; + } + + @Override + public void pushValue(T value) { + super.pushValue(value); + + Chunk chunk; + while ((chunk = getChunk()) != null) { + if (chunk.size() >= size) { + emitChunk(chunk); + } else { + // Chunk is not at full capacity yet, and neither will remaining chunks be so we can terminate. + break; + } + } + } + } + + /** + * This class represents an object which contains and manages multiple {@link rx.operators.ChunkedOperation.Chunk} objects. + * + * @param The type of objects which the internal {@link rx.operators.ChunkedOperation.Chunk} objects record. + * The type of object being tracked by the {@link Chunk} + */ + protected static class Chunks { + + private final Queue> chunks = new ConcurrentLinkedQueue>(); + private final Observer observer; + private final Func0> chunkMaker; + + /** + * Constructs a new {@link ChunkedOperation.Chunks} object for the specified {@link rx.Observer}. + * + * @param observer + * The {@link rx.Observer} to which this object will emit its internal + * {@link rx.operators.ChunkedOperation.Chunk} objects to when requested. + */ + public Chunks(Observer observer, Func0 chunkMaker) { + this.observer = observer; + this.chunkMaker = chunkMaker; + } + + /** + * This method will instantiate a new {@link rx.operators.ChunkedOperation.Chunk} object and register it internally. + * + * @return + * The constructed empty {@link rx.operators.ChunkedOperation.Chunk} object. + */ + public Chunk createChunk() { + Chunk chunk = chunkMaker.call(); + chunks.add(chunk); + return chunk; + } + + /** + * This method emits all not yet emitted {@link rx.operators.ChunkedOperation.Chunk} objects. + */ + public void emitAllChunks() { + Chunk chunk; + while ((chunk = chunks.poll()) != null) { + observer.onNext(chunk.getContents()); + } + } + + /** + * This method emits the specified {@link rx.operators.ChunkedOperation.Chunk} object. + * + * @param chunk + * The {@link rx.operators.ChunkedOperation.Chunk} to emit. + */ + public void emitChunk(Chunk chunk) { + if (!chunks.remove(chunk)) { + // Concurrency issue: Chunk is already emitted! + return; + } + observer.onNext(chunk.getContents()); + } + + /** + * @return + * The oldest (in case there are multiple) {@link rx.operators.ChunkedOperation.Chunk} object. + */ + public Chunk getChunk() { + return chunks.peek(); + } + + /** + * This method pushes a value to all not yet emitted {@link rx.operators.ChunkedOperation.Chunk} objects. + * + * @param value + * The value to push to all not yet emitted {@link rx.operators.ChunkedOperation.Chunk} objects. + */ + public void pushValue(T value) { + List> copy = new ArrayList>(chunks); + for (Chunk chunk : copy) { + chunk.pushValue(value); + } + } + } + + /** + * This {@link rx.operators.ChunkedOperation.ChunkObserver} object can be constructed using a {@link rx.operators.ChunkedOperation.Chunks} object, + * a {@link rx.Observer} object, and a {@link rx.operators.ChunkedOperation.ChunkCreator} object. The {@link rx.operators.ChunkedOperation.ChunkCreator} + * will manage the creation, and in some rare cases emission of internal {@link rx.operators.ChunkedOperation.Chunk} objects + * in the specified {@link rx.operators.ChunkedOperation.Chunks} object. Under normal circumstances the {@link rx.operators.ChunkedOperation.Chunks} + * object specifies when a created {@link rx.operators.ChunkedOperation.Chunk} is emitted. + * + * @param 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 ChunkObserver implements Observer { + + private final Chunks chunks; + private final Observer observer; + private final ChunkCreator creator; + + public ChunkObserver(Chunks chunks, Observer observer, ChunkCreator creator) { + this.observer = observer; + this.creator = creator; + this.chunks = chunks; + } + + @Override + public void onCompleted() { + creator.stop(); + chunks.emitAllChunks(); + observer.onCompleted(); + } + + @Override + public void onError(Throwable e) { + creator.stop(); + chunks.emitAllChunks(); + observer.onError(e); + } + + @Override + public void onNext(T args) { + creator.onValuePushed(); + chunks.pushValue(args); + } + } + + /** + * This {@link rx.operators.ChunkedOperation.ChunkCreator} creates a new {@link rx.operators.ChunkedOperation.Chunk} when it is initialized, but + * provides no additional functionality. This class should primarily be used when the + * internal {@link rx.operators.ChunkedOperation.Chunk} is closed externally. + * + * @param 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 SingleChunkCreator implements ChunkCreator { + + public SingleChunkCreator(Chunks chunks) { + chunks.createChunk(); + } + + @Override + public void onValuePushed() { + // Do nothing. + } + + @Override + public void stop() { + // Do nothing. + } + } + + /** + * This {@link rx.operators.ChunkedOperation.ChunkCreator} creates a new {@link rx.operators.ChunkedOperation.Chunk} whenever it receives an + * object from the provided {@link rx.Observable} created with the + * chunkClosingSelector {@link rx.util.functions.Func0}. + * + * @param 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 { + + private final SafeObservableSubscription subscription = new SafeObservableSubscription(); + private final Func0> chunkClosingSelector; + private final NonOverlappingChunks chunks; + + public ObservableBasedSingleChunkCreator(NonOverlappingChunks chunks, Func0> chunkClosingSelector) { + this.chunks = chunks; + this.chunkClosingSelector = chunkClosingSelector; + + chunks.createChunk(); + listenForChunkEnd(); + } + + private void listenForChunkEnd() { + Observable closingObservable = chunkClosingSelector.call(); + closingObservable.subscribe(new Action1() { + @Override + public void call(Closing closing) { + chunks.emitAndReplaceChunk(); + listenForChunkEnd(); + } + }); + } + + @Override + public void onValuePushed() { + // Ignore value pushes. + } + + @Override + public void stop() { + subscription.unsubscribe(); + } + } + + /** + * This {@link rx.operators.ChunkedOperation.ChunkCreator} creates a new {@link rx.operators.ChunkedOperation.Chunk} whenever it receives + * an object from the provided chunkOpenings {@link rx.Observable}, and closes the corresponding + * {@link rx.operators.ChunkedOperation.Chunk} object when it receives an object from the provided {@link rx.Observable} created + * with the chunkClosingSelector {@link rx.util.functions.Func1}. + * + * @param 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 { + + private final SafeObservableSubscription subscription = new SafeObservableSubscription(); + + public ObservableBasedMultiChunkCreator(final OverlappingChunks chunks, Observable openings, final Func1> chunkClosingSelector) { + subscription.wrap(openings.subscribe(new Action1() { + @Override + public void call(Opening opening) { + final Chunk chunk = chunks.createChunk(); + Observable closingObservable = chunkClosingSelector.call(opening); + + closingObservable.subscribe(new Action1() { + @Override + public void call(Closing closing) { + chunks.emitChunk(chunk); + } + }); + } + })); + } + + @Override + public void onValuePushed() { + // Ignore value pushes. + } + + @Override + public void stop() { + subscription.unsubscribe(); + } + } + + /** + * This {@link rx.operators.ChunkedOperation.ChunkCreator} creates a new {@link rx.operators.ChunkedOperation.Chunk} every time after a fixed + * period of time has elapsed. + * + * @param 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 TimeBasedChunkCreator implements ChunkCreator { + + private final SafeObservableSubscription subscription = new SafeObservableSubscription(); + + public TimeBasedChunkCreator(final NonOverlappingChunks chunks, long time, TimeUnit unit, Scheduler scheduler) { + this.subscription.wrap(scheduler.schedulePeriodically(new Action0() { + @Override + public void call() { + chunks.emitAndReplaceChunk(); + } + }, 0, time, unit)); + } + + public TimeBasedChunkCreator(final OverlappingChunks chunks, long time, TimeUnit unit, Scheduler scheduler) { + this.subscription.wrap(scheduler.schedulePeriodically(new Action0() { + @Override + public void call() { + chunks.createChunk(); + } + }, 0, time, unit)); + } + + @Override + public void onValuePushed() { + // Do nothing: chunks are created periodically. + } + + @Override + public void stop() { + subscription.unsubscribe(); + } + } + + /** + * This {@link rx.operators.ChunkedOperation.ChunkCreator} creates a new {@link rx.operators.ChunkedOperation.Chunk} every time after it has + * seen a certain amount of elements. + * + * @param 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 SkippingChunkCreator implements ChunkCreator { + + private final AtomicInteger skipped = new AtomicInteger(1); + private final Chunks chunks; + private final int skip; + + public SkippingChunkCreator(Chunks chunks, int skip) { + this.chunks = chunks; + this.skip = skip; + } + + @Override + public void onValuePushed() { + if (skipped.decrementAndGet() == 0) { + skipped.set(skip); + chunks.createChunk(); + } + } + + @Override + public void stop() { + // Nothing to stop: we're not using a Scheduler. + } + } +} diff --git a/rxjava-core/src/main/java/rx/operators/OperationBuffer.java b/rxjava-core/src/main/java/rx/operators/OperationBuffer.java index 05a009d715..2323f6478e 100644 --- a/rxjava-core/src/main/java/rx/operators/OperationBuffer.java +++ b/rxjava-core/src/main/java/rx/operators/OperationBuffer.java @@ -6,7 +6,7 @@ * 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. @@ -17,12 +17,7 @@ import java.util.ArrayList; import java.util.List; -import java.util.Queue; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; import org.junit.Before; import org.junit.Test; @@ -30,35 +25,40 @@ import org.mockito.Mockito; import rx.Observable; -import rx.Observable.OnSubscribeFunc; import rx.Observer; import rx.Scheduler; import rx.Subscription; import rx.concurrency.Schedulers; import rx.concurrency.TestScheduler; import rx.subscriptions.Subscriptions; -import rx.util.BufferClosing; -import rx.util.BufferClosings; -import rx.util.BufferOpening; -import rx.util.BufferOpenings; +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; import rx.util.functions.Func1; -public final class OperationBuffer { +public final class OperationBuffer extends ChunkedOperation { + + public static final Func0 BUFFER_MAKER = new Func0() { + @Override + public Object call() { + return new Buffer(); + } + }; /** *

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 BufferClosing} value. The buffer is then + * constructed using the {@link Func0} argument, produces a {@link rx.util.Closing} 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 event is propagated * to all subscribed {@link Observer}s.

* - *

Note that this operation only produces non-overlapping buffers. At all times there is + *

Note that this operation only produces non-overlapping chunks. At all times there is * exactly one buffer actively storing values.

* * @param source @@ -66,55 +66,55 @@ public final class OperationBuffer { * @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 BufferClosing} object. + * producing an {@link rx.util.Closing} object. * @return * the {@link Func1} object representing the specified buffer operation. */ - public static OnSubscribeFunc> buffer(final Observable source, final Func0> bufferClosingSelector) { - return new OnSubscribeFunc>() { + public static Func1>, Subscription> buffer(final Observable source, final Func0> bufferClosingSelector) { + return new Func1>, Subscription>() { @Override - public Subscription onSubscribe(final Observer> observer) { - NonOverlappingBuffers buffers = new NonOverlappingBuffers(observer); - BufferCreator creator = new ObservableBasedSingleBufferCreator(buffers, bufferClosingSelector); - return source.subscribe(new BufferObserver(buffers, observer, creator)); + public Subscription call(final Observer> observer) { + NonOverlappingChunks> buffers = new NonOverlappingChunks>(observer, BUFFER_MAKER); + ChunkCreator creator = new ObservableBasedSingleChunkCreator>(buffers, bufferClosingSelector); + return source.subscribe(new ChunkObserver>(buffers, observer, creator)); } }; } /** *

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 the currently active buffers. Initially - * there are no buffers active.

+ * values from the specified {@link Observable} source and stores them in the currently active chunks. Initially + * there are no chunks active.

* - *

Buffers can be created by pushing a {@link BufferOpening} value to the "bufferOpenings" {@link Observable}. + *

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 BufferClosing} 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 buffers are emitted, and the + * which can produce {@link rx.util.Closing} 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.

* - *

Note that when using this operation multiple overlapping buffers + *

Note that when using this operation multiple overlapping chunks * could be active at any one point.

* * @param source * The {@link Observable} which produces values. * @param bufferOpenings - * An {@link Observable} which when it produces a {@link BufferOpening} value will + * An {@link Observable} which when it produces a {@link rx.util.Opening} value will * 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 BufferClosing} object. + * producing an {@link rx.util.Closing} object. * @return * the {@link Func1} object representing the specified buffer operation. */ - public static OnSubscribeFunc> buffer(final Observable source, final Observable bufferOpenings, final Func1> bufferClosingSelector) { - return new OnSubscribeFunc>() { + public static Func1>, Subscription> buffer(final Observable source, final Observable bufferOpenings, final Func1> bufferClosingSelector) { + return new Func1>, Subscription>() { @Override - public Subscription onSubscribe(final Observer> observer) { - OverlappingBuffers buffers = new OverlappingBuffers(observer); - BufferCreator creator = new ObservableBasedMultiBufferCreator(buffers, bufferOpenings, bufferClosingSelector); - return source.subscribe(new BufferObserver(buffers, observer, creator)); + public Subscription call(final Observer> observer) { + OverlappingChunks> buffers = new OverlappingChunks>(observer, BUFFER_MAKER); + ChunkCreator creator = new ObservableBasedMultiChunkCreator>(buffers, bufferOpenings, bufferClosingSelector); + return source.subscribe(new ChunkObserver>(buffers, observer, creator)); } }; } @@ -126,7 +126,7 @@ public Subscription onSubscribe(final Observer> observer) { * When the source {@link Observable} completes or produces an error, the current buffer is emitted, and * the event is propagated to all subscribed {@link Observer}s.

* - *

Note that this operation only produces non-overlapping buffers. At all times there is + *

Note that this operation only produces non-overlapping chunks. At all times there is * exactly one buffer actively storing values.

* * @param source @@ -136,40 +136,40 @@ public Subscription onSubscribe(final Observer> observer) { * @return * the {@link Func1} object representing the specified buffer operation. */ - public static OnSubscribeFunc> buffer(Observable source, int count) { + public static Func1>, Subscription> buffer(Observable source, int count) { return buffer(source, count, count); } /** *

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 all active buffers until the buffer - * contains a specified number of elements. The buffer is then emitted. Buffers are created after a certain + * values from the specified {@link Observable} source and stores them in all active chunks until the buffer + * contains a specified number of elements. The buffer is then emitted. Chunks are created after a certain * amount of values have been received. When the source {@link Observable} completes or produces an error, the - * currently active buffers are emitted, and the event is propagated to all subscribed {@link Observer}s.

+ * currently active chunks are emitted, and the event is propagated to all subscribed {@link Observer}s.

* *

Note that this operation can produce non-connected, connected non-overlapping, or overlapping - * buffers depending on the input parameters.

+ * chunks depending on the input parameters.

* * @param source * The {@link Observable} which produces values. * @param count * The number of elements a buffer should have before being emitted. * @param skip - * The interval with which buffers have to be created. Note that when "skip" == "count" + * The interval with which chunks have to be created. Note that when "skip" == "count" * that this is the same as calling {@link OperationBuffer#buffer(Observable, int)}. - * If "skip" < "count", this buffer operation will produce overlapping buffers and if "skip" - * > "count" non-overlapping buffers will be created and some values will not be pushed + * If "skip" < "count", this buffer operation will produce overlapping chunks and if "skip" + * > "count" non-overlapping chunks will be created and some values will not be pushed * into a buffer at all! * @return * the {@link Func1} object representing the specified buffer operation. */ - public static OnSubscribeFunc> buffer(final Observable source, final int count, final int skip) { - return new OnSubscribeFunc>() { + public static Func1>, Subscription> buffer(final Observable source, final int count, final int skip) { + return new Func1>, Subscription>() { @Override - public Subscription onSubscribe(final Observer> observer) { - Buffers buffers = new SizeBasedBuffers(observer, count); - BufferCreator creator = new SkippingBufferCreator(buffers, skip); - return source.subscribe(new BufferObserver(buffers, observer, creator)); + public Subscription call(final Observer> observer) { + Chunks> chunks = new SizeBasedChunks>(observer, BUFFER_MAKER, count); + ChunkCreator creator = new SkippingChunkCreator>(chunks, skip); + return source.subscribe(new ChunkObserver>(chunks, observer, creator)); } }; } @@ -181,19 +181,19 @@ public Subscription onSubscribe(final Observer> observer) { * When the source {@link Observable} completes or produces an error, the current buffer is emitted, and * the event is propagated to all subscribed {@link Observer}s.

* - *

Note that this operation only produces non-overlapping buffers. At all times there is + *

Note that this operation only produces non-overlapping chunks. At all times there is * exactly one buffer actively storing values.

* * @param source * The {@link Observable} which produces values. * @param timespan - * The amount of time all buffers must be actively collect values before being emitted. + * The amount of time all chunks must be actively collect values before being emitted. * @param unit * The {@link TimeUnit} defining the unit of time for the timespan. * @return * the {@link Func1} object representing the specified buffer operation. */ - public static OnSubscribeFunc> buffer(Observable source, long timespan, TimeUnit unit) { + public static Func1>, Subscription> buffer(Observable source, long timespan, TimeUnit unit) { return buffer(source, timespan, unit, Schedulers.threadPoolForComputation()); } @@ -204,27 +204,27 @@ public static OnSubscribeFunc> buffer(Observable source * When the source {@link Observable} completes or produces an error, the current buffer is emitted, and * the event is propagated to all subscribed {@link Observer}s.

* - *

Note that this operation only produces non-overlapping buffers. At all times there is + *

Note that this operation only produces non-overlapping chunks. At all times there is * exactly one buffer actively storing values.

* * @param source * The {@link Observable} which produces values. * @param timespan - * The amount of time all buffers must be actively collect values before being emitted. + * The amount of time all chunks must be actively collect values before being emitted. * @param unit * The {@link TimeUnit} defining the unit of time for the timespan. * @param scheduler - * The {@link Scheduler} to use for timing buffers. + * The {@link Scheduler} to use for timing chunks. * @return * the {@link Func1} object representing the specified buffer operation. */ - public static OnSubscribeFunc> buffer(final Observable source, final long timespan, final TimeUnit unit, final Scheduler scheduler) { - return new OnSubscribeFunc>() { + public static Func1>, Subscription> buffer(final Observable source, final long timespan, final TimeUnit unit, final Scheduler scheduler) { + return new Func1>, Subscription>() { @Override - public Subscription onSubscribe(final Observer> observer) { - NonOverlappingBuffers buffers = new NonOverlappingBuffers(observer); - BufferCreator creator = new TimeBasedBufferCreator(buffers, timespan, unit, scheduler); - return source.subscribe(new BufferObserver(buffers, observer, creator)); + public Subscription call(final Observer> observer) { + NonOverlappingChunks> buffers = new NonOverlappingChunks>(observer, BUFFER_MAKER); + ChunkCreator creator = new TimeBasedChunkCreator>(buffers, timespan, unit, scheduler); + return source.subscribe(new ChunkObserver>(buffers, observer, creator)); } }; } @@ -237,13 +237,13 @@ public Subscription onSubscribe(final Observer> observer) { * When the source {@link Observable} completes or produces an error, the current buffer is emitted, and * the event is propagated to all subscribed {@link Observer}s.

* - *

Note that this operation only produces non-overlapping buffers. At all times there is + *

Note that this operation only produces non-overlapping chunks. At all times there is * exactly one buffer actively storing values.

* * @param source * The {@link Observable} which produces values. * @param timespan - * The amount of time all buffers must be actively collect values before being emitted. + * The amount of time all chunks must be actively collect values before being emitted. * @param unit * The {@link TimeUnit} defining the unit of time for the timespan. * @param count @@ -251,7 +251,7 @@ public Subscription onSubscribe(final Observer> observer) { * @return * the {@link Func1} object representing the specified buffer operation. */ - public static OnSubscribeFunc> buffer(Observable source, long timespan, TimeUnit unit, int count) { + public static Func1>, Subscription> buffer(Observable source, long timespan, TimeUnit unit, int count) { return buffer(source, timespan, unit, count, Schedulers.threadPoolForComputation()); } @@ -263,29 +263,29 @@ public static OnSubscribeFunc> buffer(Observable source * When the source {@link Observable} completes or produces an error, the current buffer is emitted, and * the event is propagated to all subscribed {@link Observer}s.

* - *

Note that this operation only produces non-overlapping buffers. At all times there is + *

Note that this operation only produces non-overlapping chunks. At all times there is * exactly one buffer actively storing values.

* * @param source * The {@link Observable} which produces values. * @param timespan - * The amount of time all buffers must be actively collect values before being emitted. + * The amount of time all chunks must be actively collect values before being emitted. * @param unit * The {@link TimeUnit} defining the unit of time for the timespan. * @param count * The maximum size of the buffer. Once a buffer reaches this size, it is emitted. * @param scheduler - * The {@link Scheduler} to use for timing buffers. + * The {@link Scheduler} to use for timing chunks. * @return * the {@link Func1} object representing the specified buffer operation. */ - public static OnSubscribeFunc> buffer(final Observable source, final long timespan, final TimeUnit unit, final int count, final Scheduler scheduler) { - return new OnSubscribeFunc>() { + public static Func1>, Subscription> buffer(final Observable source, final long timespan, final TimeUnit unit, final int count, final Scheduler scheduler) { + return new Func1>, Subscription>() { @Override - public Subscription onSubscribe(final Observer> observer) { - Buffers buffers = new TimeAndSizeBasedBuffers(observer, count, timespan, unit, scheduler); - BufferCreator creator = new SingleBufferCreator(buffers); - return source.subscribe(new BufferObserver(buffers, observer, creator)); + public Subscription call(final Observer> observer) { + Chunks> chunks = new TimeAndSizeBasedChunks>(observer, BUFFER_MAKER, count, timespan, unit, scheduler); + ChunkCreator creator = new SingleChunkCreator>(chunks); + return source.subscribe(new ChunkObserver>(chunks, observer, creator)); } }; } @@ -294,25 +294,25 @@ public Subscription onSubscribe(final Observer> observer) { *

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. Periodically the buffer * is emitted and replaced with a new buffer. How often this is done depends on the specified timespan. - * The creation of buffers is also periodical. How often this is done depends on the specified timeshift. + * The creation of chunks is also periodical. How often this is done depends on the specified timeshift. * When the source {@link Observable} completes or produces an error, the current buffer is emitted, and * the event is propagated to all subscribed {@link Observer}s.

* - *

Note that this operation can produce non-connected, or overlapping buffers depending + *

Note that this operation can produce non-connected, or overlapping chunks depending * on the input parameters.

* * @param source * The {@link Observable} which produces values. * @param timespan - * The amount of time all buffers must be actively collect values before being emitted. + * The amount of time all chunks must be actively collect values before being emitted. * @param timeshift - * The amount of time between creating buffers. + * The amount of time between creating chunks. * @param unit * The {@link TimeUnit} defining the unit of time for the timespan. * @return * the {@link Func1} object representing the specified buffer operation. */ - public static OnSubscribeFunc> buffer(Observable source, long timespan, long timeshift, TimeUnit unit) { + public static Func1>, Subscription> buffer(Observable source, long timespan, long timeshift, TimeUnit unit) { return buffer(source, timespan, timeshift, unit, Schedulers.threadPoolForComputation()); } @@ -320,557 +320,49 @@ public static OnSubscribeFunc> buffer(Observable source *

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. Periodically the buffer * is emitted and replaced with a new buffer. How often this is done depends on the specified timespan. - * The creation of buffers is also periodical. How often this is done depends on the specified timeshift. + * The creation of chunks is also periodical. How often this is done depends on the specified timeshift. * When the source {@link Observable} completes or produces an error, the current buffer is emitted, and * the event is propagated to all subscribed {@link Observer}s.

* - *

Note that this operation can produce non-connected, or overlapping buffers depending + *

Note that this operation can produce non-connected, or overlapping chunks depending * on the input parameters.

* * @param source * The {@link Observable} which produces values. * @param timespan - * The amount of time all buffers must be actively collect values before being emitted. + * The amount of time all chunks must be actively collect values before being emitted. * @param timeshift - * The amount of time between creating buffers. + * The amount of time between creating chunks. * @param unit * The {@link TimeUnit} defining the unit of time for the timespan. * @param scheduler - * The {@link Scheduler} to use for timing buffers. + * The {@link Scheduler} to use for timing chunks. * @return * the {@link Func1} object representing the specified buffer operation. */ - public static OnSubscribeFunc> buffer(final Observable source, final long timespan, final long timeshift, final TimeUnit unit, final Scheduler scheduler) { - return new OnSubscribeFunc>() { + public static Func1>, Subscription> buffer(final Observable source, final long timespan, final long timeshift, final TimeUnit unit, final Scheduler scheduler) { + return new Func1>, Subscription>() { @Override - public Subscription onSubscribe(final Observer> observer) { - OverlappingBuffers buffers = new TimeBasedBuffers(observer, timespan, unit, scheduler); - BufferCreator creator = new TimeBasedBufferCreator(buffers, timeshift, unit, scheduler); - return source.subscribe(new BufferObserver(buffers, observer, creator)); + public Subscription call(final Observer> observer) { + OverlappingChunks> buffers = new TimeBasedChunks>(observer, BUFFER_MAKER, timespan, unit, scheduler); + ChunkCreator creator = new TimeBasedChunkCreator>(buffers, timeshift, unit, scheduler); + return source.subscribe(new ChunkObserver>(buffers, observer, creator)); } }; } - /** - * This {@link BufferObserver} object can be constructed using a {@link Buffers} object, - * a {@link Observer} object, and a {@link BufferCreator} object. The {@link BufferCreator} - * will manage the creation, and in some rare cases emission of internal {@link Buffer} objects - * in the specified {@link Buffers} object. Under normal circumstances the {@link Buffers} - * object specifies when a created {@link Buffer} is emitted. - * - * @param The type of object all internal {@link Buffer} objects record. - */ - private static class BufferObserver implements Observer { - - private final Buffers buffers; - private final Observer> observer; - private final BufferCreator creator; - - public BufferObserver(Buffers buffers, Observer> observer, BufferCreator creator) { - this.observer = observer; - this.creator = creator; - this.buffers = buffers; - } - - @Override - public void onCompleted() { - creator.stop(); - buffers.emitAllBuffers(); - observer.onCompleted(); - } - - @Override - public void onError(Throwable e) { - creator.stop(); - buffers.emitAllBuffers(); - observer.onError(e); - } - - @Override - public void onNext(T args) { - creator.onValuePushed(); - buffers.pushValue(args); - } - } - - /** - * This interface defines a way which specifies when to create a new internal {@link Buffer} object. - * - * @param The type of object all internal {@link Buffer} objects record. - */ - private interface BufferCreator { - /** - * Signifies a onNext event. - */ - void onValuePushed(); - - /** - * Signifies a onCompleted or onError event. Should be used to clean up open - * subscriptions and other still running background tasks. - */ - void stop(); - } - - /** - * This {@link BufferCreator} creates a new {@link Buffer} when it is initialized, but - * provides no additional functionality. This class should primarily be used when the - * internal {@link Buffer} is closed externally. - * - * @param The type of object all internal {@link Buffer} objects record. - */ - private static class SingleBufferCreator implements BufferCreator { - - public SingleBufferCreator(Buffers buffers) { - buffers.createBuffer(); - } - - @Override - public void onValuePushed() { - // Do nothing. - } - - @Override - public void stop() { - // Do nothing. - } - } - - /** - * This {@link BufferCreator} creates a new {@link Buffer} whenever it receives an - * object from the provided {@link Observable} created with the - * bufferClosingSelector {@link Func0}. - * - * @param The type of object all internal {@link Buffer} objects record. - */ - private static class ObservableBasedSingleBufferCreator implements BufferCreator { - - private final SafeObservableSubscription subscription = new SafeObservableSubscription(); - private final Func0> bufferClosingSelector; - private final NonOverlappingBuffers buffers; - - public ObservableBasedSingleBufferCreator(NonOverlappingBuffers buffers, Func0> bufferClosingSelector) { - this.buffers = buffers; - this.bufferClosingSelector = bufferClosingSelector; - - buffers.createBuffer(); - listenForBufferEnd(); - } - - private void listenForBufferEnd() { - Observable closingObservable = bufferClosingSelector.call(); - closingObservable.subscribe(new Action1() { - @Override - public void call(BufferClosing closing) { - buffers.emitAndReplaceBuffer(); - listenForBufferEnd(); - } - }); - } - - @Override - public void onValuePushed() { - // Ignore value pushes. - } - - @Override - public void stop() { - subscription.unsubscribe(); - } - } - - /** - * This {@link BufferCreator} creates a new {@link Buffer} whenever it receives - * an object from the provided bufferOpenings {@link Observable}, and closes the corresponding - * {@link Buffer} object when it receives an object from the provided {@link Observable} created - * with the bufferClosingSelector {@link Func1}. - * - * @param The type of object all internal {@link Buffer} objects record. - */ - private static class ObservableBasedMultiBufferCreator implements BufferCreator { - - private final SafeObservableSubscription subscription = new SafeObservableSubscription(); - - public ObservableBasedMultiBufferCreator(final OverlappingBuffers buffers, Observable bufferOpenings, final Func1> bufferClosingSelector) { - subscription.wrap(bufferOpenings.subscribe(new Action1() { - @Override - public void call(BufferOpening opening) { - final Buffer buffer = buffers.createBuffer(); - Observable closingObservable = bufferClosingSelector.call(opening); - - closingObservable.subscribe(new Action1() { - @Override - public void call(BufferClosing closing) { - buffers.emitBuffer(buffer); - } - }); - } - })); - } - - @Override - public void onValuePushed() { - // Ignore value pushes. - } - - @Override - public void stop() { - subscription.unsubscribe(); - } - } - - /** - * This {@link BufferCreator} creates a new {@link Buffer} every time after a fixed - * period of time has elapsed. - * - * @param The type of object all internal {@link Buffer} objects record. - */ - private static class TimeBasedBufferCreator implements BufferCreator { - - private final SafeObservableSubscription subscription = new SafeObservableSubscription(); - - public TimeBasedBufferCreator(final NonOverlappingBuffers buffers, long time, TimeUnit unit, Scheduler scheduler) { - this.subscription.wrap(scheduler.schedulePeriodically(new Action0() { - @Override - public void call() { - buffers.emitAndReplaceBuffer(); - } - }, 0, time, unit)); - } - - public TimeBasedBufferCreator(final OverlappingBuffers buffers, long time, TimeUnit unit, Scheduler scheduler) { - this.subscription.wrap(scheduler.schedulePeriodically(new Action0() { - @Override - public void call() { - buffers.createBuffer(); - } - }, 0, time, unit)); - } - - @Override - public void onValuePushed() { - // Do nothing: buffers are created periodically. - } - - @Override - public void stop() { - subscription.unsubscribe(); - } - } - - /** - * This {@link BufferCreator} creates a new {@link Buffer} every time after it has - * seen a certain amount of elements. - * - * @param The type of object all internal {@link Buffer} objects record. - */ - private static class SkippingBufferCreator implements BufferCreator { - - private final AtomicInteger skipped = new AtomicInteger(1); - private final Buffers buffers; - private final int skip; - - public SkippingBufferCreator(Buffers buffers, int skip) { - this.buffers = buffers; - this.skip = skip; - } - - @Override - public void onValuePushed() { - if (skipped.decrementAndGet() == 0) { - skipped.set(skip); - buffers.createBuffer(); - } - } - - @Override - public void stop() { - // Nothing to stop: we're not using a Scheduler. - } - } - - /** - * This class is an extension on the {@link Buffers} class which only supports one - * active (not yet emitted) internal {@link Buffer} object. - * - * @param The type of object all internal {@link Buffer} objects record. - */ - private static class NonOverlappingBuffers extends Buffers { - - private final Object lock = new Object(); - - public NonOverlappingBuffers(Observer> observer) { - super(observer); - } - - public Buffer emitAndReplaceBuffer() { - synchronized (lock) { - emitBuffer(getBuffer()); - return createBuffer(); - } - } - - @Override - public void pushValue(T value) { - synchronized (lock) { - super.pushValue(value); - } - } - } - - /** - * This class is an extension on the {@link Buffers} class which actually has no additional - * behavior than its super class. Classes extending this class, are expected to support - * two or more active (not yet emitted) internal {@link Buffer} objects. - * - * @param The type of object all internal {@link Buffer} objects record. - */ - private static class OverlappingBuffers extends Buffers { - public OverlappingBuffers(Observer> observer) { - super(observer); - } - } - - /** - * This class is an extension on the {@link Buffers} class. Every internal buffer has - * a has a maximum time to live and a maximum internal capacity. When the buffer has - * reached the end of its life, or reached its maximum internal capacity it is - * automatically emitted. - * - * @param The type of object all internal {@link Buffer} objects record. - */ - private static class TimeAndSizeBasedBuffers extends Buffers { - - private final ConcurrentMap, Subscription> subscriptions = new ConcurrentHashMap, Subscription>(); - - private final Scheduler scheduler; - private final long maxTime; - private final TimeUnit unit; - private final int maxSize; - - public TimeAndSizeBasedBuffers(Observer> observer, int maxSize, long maxTime, TimeUnit unit, Scheduler scheduler) { - super(observer); - this.maxSize = maxSize; - this.maxTime = maxTime; - this.unit = unit; - this.scheduler = scheduler; - } - - @Override - public Buffer createBuffer() { - final Buffer buffer = super.createBuffer(); - subscriptions.put(buffer, scheduler.schedule(new Action0() { - @Override - public void call() { - emitBuffer(buffer); - } - }, maxTime, unit)); - return buffer; - } - - @Override - public void emitBuffer(Buffer buffer) { - Subscription subscription = subscriptions.remove(buffer); - if (subscription == null) { - // Buffer was already emitted. - return; - } - - subscription.unsubscribe(); - super.emitBuffer(buffer); - createBuffer(); - } - - @Override - public void pushValue(T value) { - super.pushValue(value); - - Buffer buffer; - while ((buffer = getBuffer()) != null) { - if (buffer.contents.size() >= maxSize) { - emitBuffer(buffer); - } else { - // Buffer is not at full capacity yet, and neither will remaining buffers be so we can terminate. - break; - } - } - } - } - - /** - * This class is an extension on the {@link Buffers} class. Every internal buffer has - * a has a maximum time to live. When the buffer has reached the end of its life it is - * automatically emitted. - * - * @param The type of object all internal {@link Buffer} objects record. - */ - private static class TimeBasedBuffers extends OverlappingBuffers { - - private final ConcurrentMap, Subscription> subscriptions = new ConcurrentHashMap, Subscription>(); - - private final Scheduler scheduler; - private final long time; - private final TimeUnit unit; - - public TimeBasedBuffers(Observer> observer, long time, TimeUnit unit, Scheduler scheduler) { - super(observer); - this.time = time; - this.unit = unit; - this.scheduler = scheduler; - } - - @Override - public Buffer createBuffer() { - final Buffer buffer = super.createBuffer(); - subscriptions.put(buffer, scheduler.schedule(new Action0() { - @Override - public void call() { - emitBuffer(buffer); - } - }, time, unit)); - return buffer; - } - - @Override - public void emitBuffer(Buffer buffer) { - subscriptions.remove(buffer); - super.emitBuffer(buffer); - } - } - - /** - * This class is an extension on the {@link Buffers} class. Every internal buffer has - * a fixed maximum capacity. When the buffer has reached its maximum capacity it is - * automatically emitted. - * - * @param The type of object all internal {@link Buffer} objects record. - */ - private static class SizeBasedBuffers extends Buffers { - - private final int size; - - public SizeBasedBuffers(Observer> observer, int size) { - super(observer); - this.size = size; - } - - @Override - public void pushValue(T value) { - super.pushValue(value); - - Buffer buffer; - while ((buffer = getBuffer()) != null) { - if (buffer.contents.size() >= size) { - emitBuffer(buffer); - } else { - // Buffer is not at full capacity yet, and neither will remaining buffers be so we can terminate. - break; - } - } - } - } - - /** - * This class represents an object which contains and manages multiple {@link Buffer} objects. - * - * @param The type of objects which the internal {@link Buffer} objects record. - */ - private static class Buffers { - - private final Queue> buffers = new ConcurrentLinkedQueue>(); - private final Observer> observer; - - /** - * Constructs a new {@link Buffers} object for the specified {@link Observer}. - * - * @param observer - * The {@link Observer} to which this object will emit its internal - * {@link Buffer} objects to when requested. - */ - public Buffers(Observer> observer) { - this.observer = observer; - } - - /** - * This method will instantiate a new {@link Buffer} object and register it internally. - * - * @return - * The constructed empty {@link Buffer} object. - */ - public Buffer createBuffer() { - Buffer buffer = new Buffer(); - buffers.add(buffer); - return buffer; - } - - /** - * This method emits all not yet emitted {@link Buffer} objects. - */ - public void emitAllBuffers() { - Buffer buffer; - while ((buffer = buffers.poll()) != null) { - observer.onNext(buffer.getContents()); - } - } - - /** - * This method emits the specified {@link Buffer} object. - * - * @param buffer - * The {@link Buffer} to emit. - */ - public void emitBuffer(Buffer buffer) { - if (!buffers.remove(buffer)) { - // Concurrency issue: Buffer is already emitted! - return; - } - observer.onNext(buffer.getContents()); - } - - /** - * @return - * The oldest (in case there are multiple) {@link Buffer} object. - */ - public Buffer getBuffer() { - return buffers.peek(); - } - - /** - * This method pushes a value to all not yet emitted {@link Buffer} objects. - * - * @param value - * The value to push to all not yet emitted {@link Buffer} objects. - */ - public void pushValue(T value) { - List> copy = new ArrayList>(buffers); - for (Buffer buffer : copy) { - buffer.pushValue(value); - } - } - } - /** * This class represents a single buffer: A sequence of recorded values. * * @param The type of objects which this {@link Buffer} can hold. */ - private static class Buffer { - private final List contents = new ArrayList(); - - /** - * Appends a specified value to the {@link Buffer}. - * - * @param value - * The value to append to the {@link Buffer}. - */ - public void pushValue(T value) { - contents.add(value); - } - + protected static class Buffer extends Chunk> { /** * @return * The mutable underlying {@link List} which contains all the * recorded values in this {@link Buffer} object. */ + @Override public List getContents() { return contents; } @@ -890,9 +382,9 @@ public void before() { @Test public void testComplete() { - Observable source = Observable.create(new OnSubscribeFunc() { + Observable source = Observable.create(new Func1, Subscription>() { @Override - public Subscription onSubscribe(Observer observer) { + public Subscription call(Observer observer) { observer.onCompleted(); return Subscriptions.empty(); } @@ -908,9 +400,9 @@ public Subscription onSubscribe(Observer observer) { @Test public void testSkipAndCountOverlappingBuffers() { - Observable source = Observable.create(new OnSubscribeFunc() { + Observable source = Observable.create(new Func1, Subscription>() { @Override - public Subscription onSubscribe(Observer observer) { + public Subscription call(Observer observer) { observer.onNext("one"); observer.onNext("two"); observer.onNext("three"); @@ -934,9 +426,9 @@ public Subscription onSubscribe(Observer observer) { @Test public void testSkipAndCountGaplessBuffers() { - Observable source = Observable.create(new OnSubscribeFunc() { + Observable source = Observable.create(new Func1, Subscription>() { @Override - public Subscription onSubscribe(Observer observer) { + public Subscription call(Observer observer) { observer.onNext("one"); observer.onNext("two"); observer.onNext("three"); @@ -960,9 +452,9 @@ public Subscription onSubscribe(Observer observer) { @Test public void testSkipAndCountBuffersWithGaps() { - Observable source = Observable.create(new OnSubscribeFunc() { + Observable source = Observable.create(new Func1, Subscription>() { @Override - public Subscription onSubscribe(Observer observer) { + public Subscription call(Observer observer) { observer.onNext("one"); observer.onNext("two"); observer.onNext("three"); @@ -986,9 +478,9 @@ public Subscription onSubscribe(Observer observer) { @Test public void testTimedAndCount() { - Observable source = Observable.create(new OnSubscribeFunc() { + Observable source = Observable.create(new Func1, Subscription>() { @Override - public Subscription onSubscribe(Observer observer) { + public Subscription call(Observer observer) { push(observer, "one", 10); push(observer, "two", 90); push(observer, "three", 110); @@ -1018,9 +510,9 @@ public Subscription onSubscribe(Observer observer) { @Test public void testTimed() { - Observable source = Observable.create(new OnSubscribeFunc() { + Observable source = Observable.create(new Func1, Subscription>() { @Override - public Subscription onSubscribe(Observer observer) { + public Subscription call(Observer observer) { push(observer, "one", 98); push(observer, "two", 99); push(observer, "three", 100); @@ -1047,9 +539,9 @@ public Subscription onSubscribe(Observer observer) { @Test public void testObservableBasedOpenerAndCloser() { - Observable source = Observable.create(new OnSubscribeFunc() { + Observable source = Observable.create(new Func1, Subscription>() { @Override - public Subscription onSubscribe(Observer observer) { + public Subscription call(Observer observer) { push(observer, "one", 10); push(observer, "two", 60); push(observer, "three", 110); @@ -1060,23 +552,23 @@ public Subscription onSubscribe(Observer observer) { } }); - Observable openings = Observable.create(new OnSubscribeFunc() { + Observable openings = Observable.create(new Func1, Subscription>() { @Override - public Subscription onSubscribe(Observer observer) { - push(observer, BufferOpenings.create(), 50); - push(observer, BufferOpenings.create(), 200); + public Subscription call(Observer observer) { + push(observer, Openings.create(), 50); + push(observer, Openings.create(), 200); complete(observer, 250); return Subscriptions.empty(); } }); - Func1> closer = new Func1>() { + Func1> closer = new Func1>() { @Override - public Observable call(BufferOpening opening) { - return Observable.create(new OnSubscribeFunc() { + public Observable call(Opening opening) { + return Observable.create(new Func1, Subscription>() { @Override - public Subscription onSubscribe(Observer observer) { - push(observer, BufferClosings.create(), 100); + public Subscription call(Observer observer) { + push(observer, Closings.create(), 100); complete(observer, 101); return Subscriptions.empty(); } @@ -1098,9 +590,9 @@ public Subscription onSubscribe(Observer observer) { @Test public void testObservableBasedCloser() { - Observable source = Observable.create(new OnSubscribeFunc() { + Observable source = Observable.create(new Func1, Subscription>() { @Override - public Subscription onSubscribe(Observer observer) { + public Subscription call(Observer observer) { push(observer, "one", 10); push(observer, "two", 60); push(observer, "three", 110); @@ -1111,13 +603,13 @@ public Subscription onSubscribe(Observer observer) { } }); - Func0> closer = new Func0>() { + Func0> closer = new Func0>() { @Override - public Observable call() { - return Observable.create(new OnSubscribeFunc() { + public Observable call() { + return Observable.create(new Func1, Subscription>() { @Override - public Subscription onSubscribe(Observer observer) { - push(observer, BufferClosings.create(), 100); + public Subscription call(Observer observer) { + push(observer, Closings.create(), 100); complete(observer, 101); return Subscriptions.empty(); } diff --git a/rxjava-core/src/main/java/rx/operators/OperationWindow.java b/rxjava-core/src/main/java/rx/operators/OperationWindow.java new file mode 100644 index 0000000000..ce603de8cf --- /dev/null +++ b/rxjava-core/src/main/java/rx/operators/OperationWindow.java @@ -0,0 +1,660 @@ +/** + * 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 rx.Observable; +import rx.Observer; +import rx.Scheduler; +import rx.Subscription; +import rx.concurrency.Schedulers; +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.*; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + + +public final class OperationWindow extends ChunkedOperation { + + public static final Func0 WINDOW_MAKER = new Func0() { + @Override + public Object call() { + return new Window(); + } + }; + + /** + *

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 + * 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 + * to all subscribed {@link rx.Observer}s.

+ * + *

Note that this operation only produces non-overlapping windows. At all times there is + * exactly one window actively storing values.

+ * + * @param source + * 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. + * @return + * the {@link rx.util.functions.Func1} object representing the specified window operation. + */ + public static Func1>, Subscription> window(final Observable source, final Func0> windowClosingSelector) { + return new Func1>, Subscription>() { + @Override + public Subscription call(final Observer> observer) { + NonOverlappingChunks> windows = new NonOverlappingChunks>(observer, WINDOW_MAKER); + ChunkCreator creator = new ObservableBasedSingleChunkCreator>(windows, windowClosingSelector); + return source.subscribe(new ChunkObserver>(windows, observer, creator)); + } + }; + } + + /** + *

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 the currently active window. Initially + * there are no windows active.

+ * + *

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 + * 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.

+ * + *

Note that when using this operation multiple overlapping windows + * could be active at any one point.

+ * + * @param source + * The {@link rx.Observable} which produces values. + * @param windowOpenings + * An {@link rx.Observable} which when it produces a {@link rx.util.Opening} value will + * 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. + * @return + * the {@link rx.util.functions.Func1} object representing the specified window operation. + */ + public static Func1>, Subscription> window(final Observable source, final Observable windowOpenings, final Func1> windowClosingSelector) { + return new Func1>, Subscription>() { + @Override + public Subscription call(final Observer> observer) { + OverlappingChunks> windows = new OverlappingChunks>(observer, WINDOW_MAKER); + ChunkCreator creator = new ObservableBasedMultiChunkCreator>(windows, windowOpenings, windowClosingSelector); + return source.subscribe(new ChunkObserver>(windows, observer, creator)); + } + }; + } + + /** + *

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 window contains + * a specified number of elements. The window is then emitted, and a new window is created to replace it. + * When the source {@link rx.Observable} completes or produces an error, the current window is emitted, and + * the event is propagated to all subscribed {@link rx.Observer}s.

+ * + *

Note that this operation only produces non-overlapping windows. At all times there is + * exactly one window actively storing values.

+ * + * @param source + * The {@link rx.Observable} which produces values. + * @param count + * The number of elements a window should have before being emitted and replaced. + * @return + * the {@link rx.util.functions.Func1} object representing the specified window operation. + */ + public static Func1>, Subscription> window(Observable source, int count) { + return window(source, count, count); + } + + /** + *

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 all active windows until the window + * contains a specified number of elements. The window is then emitted. windows are created after a certain + * amount of values have been received. When the source {@link rx.Observable} completes or produces an error, the + * currently active windows are emitted, and the event is propagated to all subscribed {@link rx.Observer}s.

+ * + *

Note that this operation can produce non-connected, connected non-overlapping, or overlapping + * windows depending on the input parameters.

+ * + * @param source + * The {@link rx.Observable} which produces values. + * @param count + * The number of elements a window should have before being emitted. + * @param skip + * The interval with which windows have to be created. Note that when "skip" == "count" + * that this is the same as calling {@link rx.operators.OperationWindow#window(rx.Observable, int)}. + * If "skip" < "count", this window operation will produce overlapping windows and if "skip" + * > "count" non-overlapping windows will be created and some values will not be pushed + * into a window at all! + * @return + * the {@link rx.util.functions.Func1} object representing the specified window operation. + */ + public static Func1>, Subscription> window(final Observable source, final int count, final int skip) { + return new Func1>, Subscription>() { + @Override + public Subscription call(final Observer> observer) { + Chunks> chunks = new SizeBasedChunks>(observer, WINDOW_MAKER, count); + ChunkCreator creator = new SkippingChunkCreator>(chunks, skip); + return source.subscribe(new ChunkObserver>(chunks, observer, creator)); + } + }; + } + + /** + *

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. Periodically the window + * is emitted and replaced with a new window. How often this is done depends on the specified timespan. + * When the source {@link rx.Observable} completes or produces an error, the current window is emitted, and + * the event is propagated to all subscribed {@link rx.Observer}s.

+ * + *

Note that this operation only produces non-overlapping windows. At all times there is + * exactly one window actively storing values.

+ * + * @param source + * The {@link rx.Observable} which produces values. + * @param timespan + * The amount of time all windows must be actively collect values before being emitted. + * @param unit + * The {@link java.util.concurrent.TimeUnit} defining the unit of time for the timespan. + * @return + * the {@link rx.util.functions.Func1} object representing the specified window operation. + */ + public static Func1>, Subscription> window(Observable source, long timespan, TimeUnit unit) { + return window(source, timespan, unit, Schedulers.threadPoolForComputation()); + } + + /** + *

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. Periodically the window + * is emitted and replaced with a new window. How often this is done depends on the specified timespan. + * When the source {@link rx.Observable} completes or produces an error, the current window is emitted, and + * the event is propagated to all subscribed {@link rx.Observer}s.

+ * + *

Note that this operation only produces non-overlapping windows. At all times there is + * exactly one window actively storing values.

+ * + * @param source + * The {@link rx.Observable} which produces values. + * @param timespan + * The amount of time all windows must be actively collect values before being emitted. + * @param unit + * The {@link java.util.concurrent.TimeUnit} defining the unit of time for the timespan. + * @param scheduler + * The {@link rx.Scheduler} to use for timing windows. + * @return + * the {@link rx.util.functions.Func1} object representing the specified window operation. + */ + public static Func1>, Subscription> window(final Observable source, final long timespan, final TimeUnit unit, final Scheduler scheduler) { + return new Func1>, Subscription>() { + @Override + public Subscription call(final Observer> observer) { + NonOverlappingChunks> windows = new NonOverlappingChunks>(observer, WINDOW_MAKER); + ChunkCreator creator = new TimeBasedChunkCreator>(windows, timespan, unit, scheduler); + return source.subscribe(new ChunkObserver>(windows, observer, creator)); + } + }; + } + + /** + *

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. Periodically the window + * is emitted and replaced with a new window. How often this is done depends on the specified timespan. + * Additionally the window is automatically emitted once it reaches a specified number of elements. + * When the source {@link rx.Observable} completes or produces an error, the current window is emitted, and + * the event is propagated to all subscribed {@link rx.Observer}s.

+ * + *

Note that this operation only produces non-overlapping windows. At all times there is + * exactly one window actively storing values.

+ * + * @param source + * The {@link rx.Observable} which produces values. + * @param timespan + * The amount of time all windows must be actively collect values before being emitted. + * @param unit + * The {@link java.util.concurrent.TimeUnit} defining the unit of time for the timespan. + * @param count + * The maximum size of the window. Once a window reaches this size, it is emitted. + * @return + * the {@link rx.util.functions.Func1} object representing the specified window operation. + */ + public static Func1>, Subscription> window(Observable source, long timespan, TimeUnit unit, int count) { + return window(source, timespan, unit, count, Schedulers.threadPoolForComputation()); + } + + /** + *

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. Periodically the window + * is emitted and replaced with a new window. How often this is done depends on the specified timespan. + * Additionally the window is automatically emitted once it reaches a specified number of elements. + * When the source {@link rx.Observable} completes or produces an error, the current window is emitted, and + * the event is propagated to all subscribed {@link rx.Observer}s.

+ * + *

Note that this operation only produces non-overlapping windows. At all times there is + * exactly one window actively storing values.

+ * + * @param source + * The {@link rx.Observable} which produces values. + * @param timespan + * The amount of time all windows must be actively collect values before being emitted. + * @param unit + * The {@link java.util.concurrent.TimeUnit} defining the unit of time for the timespan. + * @param count + * The maximum size of the window. Once a window reaches this size, it is emitted. + * @param scheduler + * The {@link rx.Scheduler} to use for timing windows. + * @return + * the {@link rx.util.functions.Func1} object representing the specified window operation. + */ + public static Func1>, Subscription> window(final Observable source, final long timespan, final TimeUnit unit, final int count, final Scheduler scheduler) { + return new Func1>, Subscription>() { + @Override + public Subscription call(final Observer> observer) { + Chunks> chunks = new TimeAndSizeBasedChunks>(observer, WINDOW_MAKER, count, timespan, unit, scheduler); + ChunkCreator creator = new SingleChunkCreator>(chunks); + return source.subscribe(new ChunkObserver>(chunks, observer, creator)); + } + }; + } + + /** + *

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. Periodically the window + * is emitted and replaced with a new window. How often this is done depends on the specified timespan. + * The creation of windows is also periodical. How often this is done depends on the specified timeshift. + * When the source {@link rx.Observable} completes or produces an error, the current window is emitted, and + * the event is propagated to all subscribed {@link rx.Observer}s.

+ * + *

Note that this operation can produce non-connected, or overlapping windows depending + * on the input parameters.

+ * + * @param source + * The {@link rx.Observable} which produces values. + * @param timespan + * The amount of time all windows must be actively collect values before being emitted. + * @param timeshift + * The amount of time between creating windows. + * @param unit + * The {@link java.util.concurrent.TimeUnit} defining the unit of time for the timespan. + * @return + * the {@link rx.util.functions.Func1} object representing the specified window operation. + */ + public static Func1>, Subscription> window(Observable source, long timespan, long timeshift, TimeUnit unit) { + return window(source, timespan, timeshift, unit, Schedulers.threadPoolForComputation()); + } + + /** + *

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. Periodically the window + * is emitted and replaced with a new window. How often this is done depends on the specified timespan. + * The creation of windows is also periodical. How often this is done depends on the specified timeshift. + * When the source {@link rx.Observable} completes or produces an error, the current window is emitted, and + * the event is propagated to all subscribed {@link rx.Observer}s.

+ * + *

Note that this operation can produce non-connected, or overlapping windows depending + * on the input parameters.

+ * + * @param source + * The {@link rx.Observable} which produces values. + * @param timespan + * The amount of time all windows must be actively collect values before being emitted. + * @param timeshift + * The amount of time between creating windows. + * @param unit + * The {@link java.util.concurrent.TimeUnit} defining the unit of time for the timespan. + * @param scheduler + * The {@link rx.Scheduler} to use for timing windows. + * @return + * the {@link rx.util.functions.Func1} object representing the specified window operation. + */ + public static Func1>, Subscription> window(final Observable source, final long timespan, final long timeshift, final TimeUnit unit, final Scheduler scheduler) { + return new Func1>, Subscription>() { + @Override + public Subscription call(final Observer> observer) { + OverlappingChunks> windows = new TimeBasedChunks>(observer, WINDOW_MAKER, timespan, unit, scheduler); + ChunkCreator creator = new TimeBasedChunkCreator>(windows, timeshift, unit, scheduler); + return source.subscribe(new ChunkObserver>(windows, observer, creator)); + } + }; + } + + /** + * This class represents a single window: A sequence of recorded values. + * + * @param The type of objects which this {@link Window} can hold. + */ + protected static class Window extends Chunk> { + /** + * @return + * The mutable underlying {@link Observable} which contains all the + * recorded values in this {@link Window} object. + */ + @Override + public Observable getContents() { + return Observable.from(contents); + } + } + + public static class UnitTest { + + private TestScheduler scheduler; + + @Before + @SuppressWarnings("unchecked") + public void before() { + scheduler = new TestScheduler(); + } + + private static List> toLists(Observable> observable) { + final List list = new ArrayList(); + final List> lists = new ArrayList>(); + + observable.subscribe(new Action1>() { + @Override + public void call(Observable tObservable) { + tObservable.subscribe(new Action1() { + @Override + public void call(T t) { + list.add(t); + } + }); + lists.add(new ArrayList(list)); + list.clear(); + } + }); + return lists; + } + + @Test + public void testNonOverlappingWindows() { + Observable subject = Observable.from("one", "two", "three", "four", "five"); + Observable> windowed = Observable.create(window(subject, 3)); + + List> windows = toLists(windowed); + + assertEquals(2, windows.size()); + assertEquals(list("one", "two", "three"), windows.get(0)); + assertEquals(list("four", "five"), windows.get(1)); + } + + @Test + public void testSkipAndCountGaplessEindows() { + Observable subject = Observable.from("one", "two", "three", "four", "five"); + Observable> windowed = Observable.create(window(subject, 3, 3)); + + List> windows = toLists(windowed); + + assertEquals(2, windows.size()); + assertEquals(list("one", "two", "three"), windows.get(0)); + assertEquals(list("four", "five"), windows.get(1)); + } + + @Test + public void testOverlappingWindows() { + Observable subject = Observable.from("zero", "one", "two", "three", "four", "five"); + Observable> windowed = Observable.create(window(subject, 3, 1)); + + List> windows = toLists(windowed); + + assertEquals(6, windows.size()); + assertEquals(list("zero", "one", "two"), windows.get(0)); + assertEquals(list("one", "two", "three"), windows.get(1)); + assertEquals(list("two", "three", "four"), windows.get(2)); + assertEquals(list("three", "four", "five"), windows.get(3)); + assertEquals(list("four", "five"), windows.get(4)); + assertEquals(list("five"), windows.get(5)); + } + + @Test + public void testSkipAndCountWindowsWithGaps() { + Observable subject = Observable.from("one", "two", "three", "four", "five"); + Observable> windowed = Observable.create(window(subject, 2, 3)); + + List> windows = toLists(windowed); + + assertEquals(2, windows.size()); + assertEquals(list("one", "two"), windows.get(0)); + assertEquals(list("four", "five"), windows.get(1)); + } + + @Test + public void testTimedAndCount() { + final List list = new ArrayList(); + final List> lists = new ArrayList>(); + + Observable source = Observable.create(new Func1, Subscription>() { + @Override + public Subscription call(Observer observer) { + push(observer, "one", 10); + push(observer, "two", 90); + push(observer, "three", 110); + push(observer, "four", 190); + push(observer, "five", 210); + complete(observer, 250); + return Subscriptions.empty(); + } + }); + + Observable> windowed = Observable.create(window(source, 100, TimeUnit.MILLISECONDS, 2, scheduler)); + windowed.subscribe(observeWindow(list, lists)); + + scheduler.advanceTimeTo(100, TimeUnit.MILLISECONDS); + assertEquals(1, lists.size()); + assertEquals(lists.get(0), list("one", "two")); + + scheduler.advanceTimeTo(200, TimeUnit.MILLISECONDS); + assertEquals(2, lists.size()); + assertEquals(lists.get(1), list("three", "four")); + + scheduler.advanceTimeTo(300, TimeUnit.MILLISECONDS); + assertEquals(3, lists.size()); + assertEquals(lists.get(2), list("five")); + } + + @Test + public void testTimed() { + final List list = new ArrayList(); + final List> lists = new ArrayList>(); + + Observable source = Observable.create(new Func1, Subscription>() { + @Override + public Subscription call(Observer observer) { + push(observer, "one", 98); + push(observer, "two", 99); + push(observer, "three", 100); + push(observer, "four", 101); + push(observer, "five", 102); + complete(observer, 150); + return Subscriptions.empty(); + } + }); + + Observable> windowed = Observable.create(window(source, 100, TimeUnit.MILLISECONDS, scheduler)); + windowed.subscribe(observeWindow(list, lists)); + + scheduler.advanceTimeTo(101, TimeUnit.MILLISECONDS); + assertEquals(1, lists.size()); + assertEquals(lists.get(0), list("one", "two", "three")); + + scheduler.advanceTimeTo(201, TimeUnit.MILLISECONDS); + assertEquals(2, lists.size()); + assertEquals(lists.get(1), list("four", "five")); + } + + @Test + public void testObservableBasedOpenerAndCloser() { + final List list = new ArrayList(); + final List> lists = new ArrayList>(); + + Observable source = Observable.create(new Func1, Subscription>() { + @Override + public Subscription call(Observer observer) { + push(observer, "one", 10); + push(observer, "two", 60); + push(observer, "three", 110); + push(observer, "four", 160); + push(observer, "five", 210); + complete(observer, 500); + return Subscriptions.empty(); + } + }); + + Observable openings = Observable.create(new Func1, Subscription>() { + @Override + public Subscription call(Observer observer) { + push(observer, Openings.create(), 50); + push(observer, Openings.create(), 200); + complete(observer, 250); + return Subscriptions.empty(); + } + }); + + Func1> closer = new Func1>() { + @Override + public Observable call(Opening opening) { + return Observable.create(new Func1, Subscription>() { + @Override + public Subscription call(Observer observer) { + push(observer, Closings.create(), 100); + complete(observer, 101); + return Subscriptions.empty(); + } + }); + } + }; + + Observable> windowed = Observable.create(window(source, openings, closer)); + windowed.subscribe(observeWindow(list, lists)); + + scheduler.advanceTimeTo(500, TimeUnit.MILLISECONDS); + assertEquals(2, lists.size()); + assertEquals(lists.get(0), list("two", "three")); + assertEquals(lists.get(1), list("five")); + } + + @Test + public void testObservableBasedCloser() { + final List list = new ArrayList(); + final List> lists = new ArrayList>(); + + Observable source = Observable.create(new Func1, Subscription>() { + @Override + public Subscription call(Observer observer) { + push(observer, "one", 10); + push(observer, "two", 60); + push(observer, "three", 110); + push(observer, "four", 160); + push(observer, "five", 210); + complete(observer, 250); + return Subscriptions.empty(); + } + }); + + Func0> closer = new Func0>() { + @Override + public Observable call() { + return Observable.create(new Func1, Subscription>() { + @Override + public Subscription call(Observer observer) { + push(observer, Closings.create(), 100); + complete(observer, 101); + return Subscriptions.empty(); + } + }); + } + }; + + Observable> windowed = Observable.create(window(source, closer)); + windowed.subscribe(observeWindow(list, lists)); + + scheduler.advanceTimeTo(500, TimeUnit.MILLISECONDS); + assertEquals(3, lists.size()); + assertEquals(lists.get(0), list("one", "two")); + assertEquals(lists.get(1), list("three", "four")); + assertEquals(lists.get(2), list("five")); + } + + private List list(String... args) { + List list = new ArrayList(); + for (String arg : args) { + list.add(arg); + } + return list; + } + + private void push(final Observer observer, final T value, int delay) { + scheduler.schedule(new Action0() { + @Override + public void call() { + observer.onNext(value); + } + }, delay, TimeUnit.MILLISECONDS); + } + + private void complete(final Observer observer, int delay) { + scheduler.schedule(new Action0() { + @Override + public void call() { + observer.onCompleted(); + } + }, delay, TimeUnit.MILLISECONDS); + } + + private Action1> observeWindow(final List list, final List> lists) { + return new Action1>() { + @Override + public void call(Observable stringObservable) { + stringObservable.subscribe(new Observer() { + @Override + public void onCompleted() { + lists.add(new ArrayList(list)); + list.clear(); + } + + @Override + public void onError(Throwable e) { + fail(e.getMessage()); + } + + @Override + public void onNext(String args) { + list.add(args); + } + }); + } + }; + } + + } +} diff --git a/rxjava-core/src/main/java/rx/util/BufferClosing.java b/rxjava-core/src/main/java/rx/util/Closing.java similarity index 95% rename from rxjava-core/src/main/java/rx/util/BufferClosing.java rename to rxjava-core/src/main/java/rx/util/Closing.java index 8340e2f36d..987d7a9dbe 100644 --- a/rxjava-core/src/main/java/rx/util/BufferClosing.java +++ b/rxjava-core/src/main/java/rx/util/Closing.java @@ -15,6 +15,6 @@ */ package rx.util; -public interface BufferClosing { +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/BufferClosings.java b/rxjava-core/src/main/java/rx/util/Closings.java similarity index 82% rename from rxjava-core/src/main/java/rx/util/BufferClosings.java rename to rxjava-core/src/main/java/rx/util/Closings.java index 4a6891e917..cc6c589283 100644 --- a/rxjava-core/src/main/java/rx/util/BufferClosings.java +++ b/rxjava-core/src/main/java/rx/util/Closings.java @@ -15,13 +15,13 @@ */ package rx.util; -public class BufferClosings { +public class Closings { - public static BufferClosing create() { - return new BufferClosing() {}; + public static Closing create() { + return new Closing() {}; } - private BufferClosings() { + private Closings() { // Prevent instantation. } } \ No newline at end of file diff --git a/rxjava-core/src/main/java/rx/util/BufferOpening.java b/rxjava-core/src/main/java/rx/util/Opening.java similarity index 95% rename from rxjava-core/src/main/java/rx/util/BufferOpening.java rename to rxjava-core/src/main/java/rx/util/Opening.java index 847dbbb947..03a0bbcfb1 100644 --- a/rxjava-core/src/main/java/rx/util/BufferOpening.java +++ b/rxjava-core/src/main/java/rx/util/Opening.java @@ -15,6 +15,6 @@ */ package rx.util; -public interface BufferOpening { +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/BufferOpenings.java b/rxjava-core/src/main/java/rx/util/Openings.java similarity index 82% rename from rxjava-core/src/main/java/rx/util/BufferOpenings.java rename to rxjava-core/src/main/java/rx/util/Openings.java index 59482d0556..3f962ec163 100644 --- a/rxjava-core/src/main/java/rx/util/BufferOpenings.java +++ b/rxjava-core/src/main/java/rx/util/Openings.java @@ -15,13 +15,13 @@ */ package rx.util; -public class BufferOpenings { +public class Openings { - public static BufferOpening create() { - return new BufferOpening() {}; + public static Opening create() { + return new Opening() {}; } - private BufferOpenings() { + private Openings() { // Prevent instantation. } } \ No newline at end of file From 09cac8302d0bb6277e6bcbcb69ea677abe515684 Mon Sep 17 00:00:00 2001 From: Ben Christensen Date: Wed, 4 Sep 2013 23:19:10 -0700 Subject: [PATCH 2/5] Merge branch 'OperationWindow' of https://github.com/RallySoftware/RxJava - updated to use new OnSubscribeFunc - updated to use super/extends generics --- rxjava-core/src/main/java/rx/Observable.java | 528 +++++++++++++----- .../java/rx/operators/ChunkedOperation.java | 256 +++++---- .../java/rx/operators/OperationBuffer.java | 183 +++--- .../java/rx/operators/OperationWindow.java | 190 +++---- 4 files changed, 704 insertions(+), 453 deletions(-) diff --git a/rxjava-core/src/main/java/rx/Observable.java b/rxjava-core/src/main/java/rx/Observable.java index 17b54c7039..25073ca729 100644 --- a/rxjava-core/src/main/java/rx/Observable.java +++ b/rxjava-core/src/main/java/rx/Observable.java @@ -73,8 +73,8 @@ import rx.subjects.Subject; import rx.subscriptions.Subscriptions; import rx.util.Closing; -import rx.util.Opening; import rx.util.OnErrorNotImplementedException; +import rx.util.Opening; import rx.util.Range; import rx.util.Timestamped; import rx.util.functions.Action0; @@ -84,6 +84,11 @@ 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; import rx.util.functions.Function; @@ -98,18 +103,27 @@ *

* *

- * For more information see the RxJava - * Wiki + * For more information see the RxJava Wiki * * @param */ public class Observable { - //TODO use a consistent parameter naming scheme (for example: for all operators that modify a source Observable, the parameter representing that source Observable should have the same name, e.g. "source" -- currently such parameters are named any of "sequence", "that", "source", "items", or "observable") + /** + * Executed when 'subscribe' is invoked. + */ + private final OnSubscribeFunc onSubscribe; - private final static RxJavaObservableExecutionHook hook = RxJavaPlugins.getInstance().getObservableExecutionHook(); + /** + * Function interface for work to be performed when an {@link Observable} is subscribed to via {@link Observable#subscribe(Observer)} + * + * @param + */ + public static interface OnSubscribeFunc extends Function { - private final Func1, Subscription> onSubscribe; + public Subscription onSubscribe(Observer t1); + + } /** * Observable with Function to execute when subscribed to. @@ -118,16 +132,13 @@ public class Observable { * specifically have a need for inheritance. * * @param onSubscribe - * {@link Func1} to be executed when {@link #subscribe(Observer)} is called. + * {@link OnSubscribeFunc} to be executed when {@link #subscribe(Observer)} is called. */ - protected Observable(Func1, Subscription> onSubscribe) { + protected Observable(OnSubscribeFunc onSubscribe) { this.onSubscribe = onSubscribe; } - protected Observable() { - this(null); - //TODO should this be made private to prevent it? It really serves no good purpose and only confuses things. Unit tests are incorrectly using it today - } + private final static RxJavaObservableExecutionHook hook = RxJavaPlugins.getInstance().getObservableExecutionHook(); /** * An {@link Observer} must call an Observable's {@code subscribe} method in order to @@ -156,9 +167,9 @@ protected Observable() { * @throws IllegalArgumentException * if the {@link Observer} provided as the argument to {@code subscribe()} is {@code null} */ - public Subscription subscribe(Observer observer) { + public Subscription subscribe(Observer observer) { // allow the hook to intercept and/or decorate - Func1, Subscription> onSubscribeFunction = hook.onSubscribeStart(this, onSubscribe); + OnSubscribeFunc onSubscribeFunction = hook.onSubscribeStart(this, onSubscribe); // validate and proceed if (observer == null) { throw new IllegalArgumentException("observer can not be null"); @@ -172,7 +183,7 @@ public Subscription subscribe(Observer observer) { * See https://github.com/Netflix/RxJava/issues/216 for discussion on "Guideline 6.4: Protect calls to user code from within an operator" */ if (isInternalImplementation(observer)) { - Subscription s = onSubscribeFunction.call(observer); + Subscription s = onSubscribeFunction.onSubscribe(observer); if (s == null) { // this generally shouldn't be the case on a 'trusted' onSubscribe but in case it happens // we want to gracefully handle it the same as AtomicObservableSubscription does @@ -182,7 +193,7 @@ public Subscription subscribe(Observer observer) { } } else { SafeObservableSubscription subscription = new SafeObservableSubscription(); - subscription.wrap(onSubscribeFunction.call(new SafeObserver(subscription, observer))); + subscription.wrap(onSubscribeFunction.onSubscribe(new SafeObserver(subscription, observer))); return hook.onSubscribeReturn(this, subscription); } } catch (OnErrorNotImplementedException e) { @@ -234,7 +245,7 @@ public Subscription subscribe(Observer observer) { * @throws IllegalArgumentException * if an argument to {@code subscribe()} is {@code null} */ - public Subscription subscribe(Observer observer, Scheduler scheduler) { + public Subscription subscribe(Observer observer, Scheduler scheduler) { return subscribeOn(scheduler).subscribe(observer); } @@ -243,12 +254,12 @@ public Subscription subscribe(Observer observer, Scheduler scheduler) { *

* See https://github.com/Netflix/RxJava/issues/216 for discussion on "Guideline 6.4: Protect calls to user code from within an operator" */ - private Subscription protectivelyWrapAndSubscribe(Observer o) { + private Subscription protectivelyWrapAndSubscribe(Observer o) { SafeObservableSubscription subscription = new SafeObservableSubscription(); return subscription.wrap(subscribe(new SafeObserver(subscription, o))); } - public Subscription subscribe(final Action1 onNext) { + public Subscription subscribe(final Action1 onNext) { if (onNext == null) { throw new IllegalArgumentException("onNext can not be null"); } @@ -279,11 +290,11 @@ public void onNext(T args) { }); } - public Subscription subscribe(final Action1 onNext, Scheduler scheduler) { + public Subscription subscribe(final Action1 onNext, Scheduler scheduler) { return subscribeOn(scheduler).subscribe(onNext); } - public Subscription subscribe(final Action1 onNext, final Action1 onError) { + public Subscription subscribe(final Action1 onNext, final Action1 onError) { if (onNext == null) { throw new IllegalArgumentException("onNext can not be null"); } @@ -317,11 +328,11 @@ public void onNext(T args) { }); } - public Subscription subscribe(final Action1 onNext, final Action1 onError, Scheduler scheduler) { + public Subscription subscribe(final Action1 onNext, final Action1 onError, Scheduler scheduler) { return subscribeOn(scheduler).subscribe(onNext, onError); } - public Subscription subscribe(final Action1 onNext, final Action1 onError, final Action0 onComplete) { + public Subscription subscribe(final Action1 onNext, final Action1 onError, final Action0 onComplete) { if (onNext == null) { throw new IllegalArgumentException("onNext can not be null"); } @@ -358,7 +369,7 @@ public void onNext(T args) { }); } - public Subscription subscribe(final Action1 onNext, final Action1 onError, final Action0 onComplete, Scheduler scheduler) { + public Subscription subscribe(final Action1 onNext, final Action1 onError, final Action0 onComplete, Scheduler scheduler) { return subscribeOn(scheduler).subscribe(onNext, onError, onComplete); } @@ -398,10 +409,10 @@ private void handleError(Throwable e) { */ private static class NeverObservable extends Observable { public NeverObservable() { - super(new Func1, Subscription>() { + super(new OnSubscribeFunc() { @Override - public Subscription call(Observer t1) { + public Subscription onSubscribe(Observer t1) { return Subscriptions.empty(); } @@ -418,7 +429,7 @@ public Subscription call(Observer t1) { private static class ThrowObservable extends Observable { public ThrowObservable(final Throwable exception) { - super(new Func1, Subscription>() { + super(new OnSubscribeFunc() { /** * Accepts an {@link Observer} and calls its {@link Observer#onError onError} method. @@ -428,7 +439,7 @@ public ThrowObservable(final Throwable exception) { * @return a reference to the subscription */ @Override - public Subscription call(Observer observer) { + public Subscription onSubscribe(Observer observer) { observer.onError(exception); return Subscriptions.empty(); } @@ -462,7 +473,7 @@ public Subscription call(Observer observer) { * @return an Observable that, when an {@link Observer} subscribes to it, will execute the given * function */ - public static Observable create(Func1, Subscription> func) { + public static Observable create(OnSubscribeFunc func) { return new Observable(func); } @@ -511,7 +522,7 @@ public static Observable error(Throwable exception) { * emitted by the resulting Observable * @return an Observable that emits each item in the source {@link Iterable} sequence */ - public static Observable from(Iterable iterable) { + public static Observable from(Iterable iterable) { return create(OperationToObservableIterable.toObservableIterable(iterable)); } @@ -575,7 +586,7 @@ public static Observable range(int start, int count) { * @return an Observable whose {@link Observer}s trigger an invocation of the given Observable * factory function */ - public static Observable defer(Func0> observableFactory) { + public static Observable defer(Func0> observableFactory) { return create(OperationDefer.defer(observableFactory)); } @@ -618,7 +629,7 @@ public static Observable just(T value) { * @return an Observable that emits items that are the result of flattening the {@code source} list of Observables * @see MSDN: Observable.Merge */ - public static Observable merge(List> source) { + public static Observable merge(List> source) { return create(OperationMerge.merge(source)); } @@ -637,7 +648,7 @@ public static Observable merge(List> source) { * by the Observables emitted by the {@code source} Observable * @see MSDN: Observable.Merge Method */ - public static Observable merge(Observable> source) { + public static Observable merge(Observable> source) { return create(OperationMerge.merge(source)); } @@ -655,7 +666,7 @@ public static Observable merge(Observable> source) { * by the {@code source} Observables * @see MSDN: Observable.Merge Method */ - public static Observable merge(Observable... source) { + public static Observable merge(Observable... source) { return create(OperationMerge.merge(source)); } @@ -671,7 +682,7 @@ public static Observable merge(Observable... source) { * the {@code source} Observables, one after the other * @see MSDN: Observable.Concat Method */ - public static Observable concat(Observable... source) { + public static Observable concat(Observable... source) { return create(OperationConcat.concat(source)); } @@ -695,7 +706,7 @@ public static Observable concat(Observable... source) { * the {@code source} list of Observables * @see MSDN: Observable.Merge Method */ - public static Observable mergeDelayError(List> source) { + public static Observable mergeDelayError(List> source) { return create(OperationMergeDelayError.mergeDelayError(source)); } @@ -719,7 +730,7 @@ public static Observable mergeDelayError(List> source) { * the Observables emitted by the {@code source} Observable * @see MSDN: Observable.Merge Method */ - public static Observable mergeDelayError(Observable> source) { + public static Observable mergeDelayError(Observable> source) { return create(OperationMergeDelayError.mergeDelayError(source)); } @@ -743,7 +754,7 @@ public static Observable mergeDelayError(Observable> source * the {@code source} Observables * @see MSDN: Observable.Merge Method */ - public static Observable mergeDelayError(Observable... source) { + public static Observable mergeDelayError(Observable... source) { return create(OperationMergeDelayError.mergeDelayError(source)); } @@ -773,7 +784,7 @@ public static Observable never() { * @return an Observable that emits only the items emitted by the most recently published * Observable */ - public static Observable switchDo(Observable> sequenceOfSequences) { + public static Observable switchDo(Observable> sequenceOfSequences) { // TODO should this static remain? I have left it because it is an Observable return create(OperationSwitch.switchDo(sequenceOfSequences)); } @@ -793,7 +804,7 @@ public static Observable switchDo(Observable> sequenceOfSeq public Observable switchDo() { // TODO can we come up with a better name than this? It should be 'switch' but that is reserved. // Perhaps 'switchOnNext'? - return create(OperationSwitch.switchDo((Observable>) this)); + return create(OperationSwitch.switchDo((Observable>) this)); } /** @@ -813,7 +824,7 @@ public Observable switchDo() { * @return an Observable that is a chronologically well-behaved version of the source * Observable, and that synchronously notifies its {@link Observer}s */ - public static Observable synchronize(Observable observable) { + public static Observable synchronize(Observable observable) { return create(OperationSynchronize.synchronize(observable)); } @@ -846,7 +857,7 @@ public Observable> timestamp() { * be emitted by the resulting Observable * @return an Observable that emits the item from the source Future */ - public static Observable from(Future future) { + public static Observable from(Future future) { return create(OperationToObservableFuture.toObservableFuture(future)); } @@ -869,7 +880,7 @@ public static Observable from(Future future) { * be emitted by the resulting Observable * @return an Observable that emits the item from the source Future */ - public static Observable from(Future future, Scheduler scheduler) { + public static Observable from(Future future, Scheduler scheduler) { return create(OperationToObservableFuture.toObservableFuture(future)).subscribeOn(scheduler); } @@ -895,7 +906,7 @@ public static Observable from(Future future, Scheduler scheduler) { * be emitted by the resulting Observable * @return an Observable that emits the item from the source {@link Future} */ - public static Observable from(Future future, long timeout, TimeUnit unit) { + public static Observable from(Future future, long timeout, TimeUnit unit) { return create(OperationToObservableFuture.toObservableFuture(future, timeout, unit)); } @@ -909,18 +920,17 @@ public static Observable from(Future future, long timeout, TimeUnit un * The resulting {@code Observable} returned from {@code zip} will invoke {@link Observer#onNext onNext} as many times as the number of {@code onNext} invocations * of the source Observable that emits the fewest items. * - * @param w0 + * @param o1 * one source Observable - * @param w1 + * @param o2 * another source Observable - * @param reduceFunction - * a function that, when applied to a pair of items, each emitted by one of the two - * source Observables, results in an item that will be emitted by the resulting - * Observable + * @param zipFunction + * a function that, when applied to an item emitted by each of the source + * Observables, results in an item that will be emitted by the resulting Observable * @return an Observable that emits the zipped results */ - public static Observable zip(Observable w0, Observable w1, Func2 reduceFunction) { - return create(OperationZip.zip(w0, w1, reduceFunction)); + public static Observable zip(Observable o1, Observable o2, Func2 zipFunction) { + return create(OperationZip.zip(o1, o2, zipFunction)); } /** @@ -938,7 +948,7 @@ public static Observable zip(Observable w0, Observable w1 * @return an Observable that emits Booleans that indicate whether the corresponding items * emitted by the source Observables are equal */ - public static Observable sequenceEqual(Observable first, Observable second) { + public static Observable sequenceEqual(Observable first, Observable second) { return sequenceEqual(first, second, new Func2() { @Override public Boolean call(T first, T second) { @@ -965,7 +975,7 @@ public Boolean call(T first, T second) { * @return an Observable that emits Booleans that indicate whether the corresponding items * emitted by the source Observables are equal */ - public static Observable sequenceEqual(Observable first, Observable second, Func2 equality) { + public static Observable sequenceEqual(Observable first, Observable second, Func2 equality) { return zip(first, second, equality); } @@ -982,19 +992,19 @@ public static Observable sequenceEqual(Observable first, Observa * The resulting {@code Observable} returned from {@code zip} will invoke {@link Observer#onNext onNext} as many times as the number of {@code onNext} invocations * of the source Observable that emits the fewest items. * - * @param w0 + * @param o1 * one source Observable - * @param w1 - * another source Observable - * @param w2 + * @param o2 + * a second source Observable + * @param o3 * a third source Observable - * @param function + * @param zipFunction * a function that, when applied to an item emitted by each of the source * Observables, results in an item that will be emitted by the resulting Observable * @return an Observable that emits the zipped results */ - public static Observable zip(Observable w0, Observable w1, Observable w2, Func3 function) { - return create(OperationZip.zip(w0, w1, w2, function)); + public static Observable zip(Observable o1, Observable o2, Observable o3, Func3 zipFunction) { + return create(OperationZip.zip(o1, o2, o3, zipFunction)); } /** @@ -1011,21 +1021,210 @@ public static Observable zip(Observable w0, Observable} returned from {@code zip} will invoke {@link Observer#onNext onNext} as many times as the number of {@code onNext} invocations * of the source Observable that emits the fewest items. * - * @param w0 + * @param o1 * one source Observable - * @param w1 - * another source Observable - * @param w2 + * @param o2 + * a second source Observable + * @param o3 + * a third source Observable + * @param o4 + * a fourth source Observable + * @param zipFunction + * a function that, when applied to an item emitted by each of the source + * Observables, results in an item that will be emitted by the resulting Observable + * @return an Observable that emits the zipped results + */ + public static Observable zip(Observable o1, Observable o2, Observable o3, Observable o4, Func4 zipFunction) { + return create(OperationZip.zip(o1, o2, o3, o4, zipFunction)); + } + + /** + * Returns an Observable that emits the results of a function of your choosing applied to + * combinations of four items emitted, in sequence, by four other Observables. + *

+ * + *

{@code zip} applies this function in strict sequence, so the first item emitted by the + * new Observable will be the result of the function applied to the first item emitted by {@code w0}, the first item emitted by {@code w1}, the first item emitted by {@code w2}, and the first item + * emitted by {@code w3}; the second item emitted by + * the new Observable will be the result of the function applied to the second item emitted by + * each of those Observables; and so forth. + *

+ * The resulting {@code Observable} returned from {@code zip} will invoke {@link Observer#onNext onNext} as many times as the number of {@code onNext} invocations + * of the source Observable that emits the fewest items. + * + * @param o1 + * one source Observable + * @param o2 + * a second source Observable + * @param o3 + * a third source Observable + * @param o4 + * a fourth source Observable + * @param o5 + * a fifth source Observable + * @param zipFunction + * a function that, when applied to an item emitted by each of the source + * Observables, results in an item that will be emitted by the resulting Observable + * @return an Observable that emits the zipped results + */ + public static Observable zip(Observable o1, Observable o2, Observable o3, Observable o4, Observable o5, Func5 zipFunction) { + return create(OperationZip.zip(o1, o2, o3, o4, o5, zipFunction)); + } + + /** + * Returns an Observable that emits the results of a function of your choosing applied to + * combinations of four items emitted, in sequence, by four other Observables. + *

+ * + *

{@code zip} applies this function in strict sequence, so the first item emitted by the + * new Observable will be the result of the function applied to the first item emitted by {@code w0}, the first item emitted by {@code w1}, the first item emitted by {@code w2}, and the first item + * emitted by {@code w3}; the second item emitted by + * the new Observable will be the result of the function applied to the second item emitted by + * each of those Observables; and so forth. + *

+ * The resulting {@code Observable} returned from {@code zip} will invoke {@link Observer#onNext onNext} as many times as the number of {@code onNext} invocations + * of the source Observable that emits the fewest items. + * + * @param o1 + * one source Observable + * @param o2 + * a second source Observable + * @param o3 + * a third source Observable + * @param o4 + * a fourth source Observable + * @param o5 + * a fifth source Observable + * @param o6 + * a sixth source Observable + * @param zipFunction + * a function that, when applied to an item emitted by each of the source + * Observables, results in an item that will be emitted by the resulting Observable + * @return an Observable that emits the zipped results + */ + public static Observable zip(Observable o1, Observable o2, Observable o3, Observable o4, Observable o5, Observable o6, + Func6 zipFunction) { + return create(OperationZip.zip(o1, o2, o3, o4, o5, o6, zipFunction)); + } + + /** + * Returns an Observable that emits the results of a function of your choosing applied to + * combinations of four items emitted, in sequence, by four other Observables. + *

+ * + *

{@code zip} applies this function in strict sequence, so the first item emitted by the + * new Observable will be the result of the function applied to the first item emitted by {@code w0}, the first item emitted by {@code w1}, the first item emitted by {@code w2}, and the first item + * emitted by {@code w3}; the second item emitted by + * the new Observable will be the result of the function applied to the second item emitted by + * each of those Observables; and so forth. + *

+ * The resulting {@code Observable} returned from {@code zip} will invoke {@link Observer#onNext onNext} as many times as the number of {@code onNext} invocations + * of the source Observable that emits the fewest items. + * + * @param o1 + * one source Observable + * @param o2 + * a second source Observable + * @param o3 * a third source Observable - * @param w3 + * @param o4 * a fourth source Observable - * @param reduceFunction + * @param o5 + * a fifth source Observable + * @param o6 + * a sixth source Observable + * @param o7 + * a seventh source Observable + * @param zipFunction * a function that, when applied to an item emitted by each of the source * Observables, results in an item that will be emitted by the resulting Observable * @return an Observable that emits the zipped results */ - public static Observable zip(Observable w0, Observable w1, Observable w2, Observable w3, Func4 reduceFunction) { - return create(OperationZip.zip(w0, w1, w2, w3, reduceFunction)); + public static Observable zip(Observable o1, Observable o2, Observable o3, Observable o4, Observable o5, Observable o6, Observable o7, + Func7 zipFunction) { + return create(OperationZip.zip(o1, o2, o3, o4, o5, o6, o7, zipFunction)); + } + + /** + * Returns an Observable that emits the results of a function of your choosing applied to + * combinations of four items emitted, in sequence, by four other Observables. + *

+ * + *

{@code zip} applies this function in strict sequence, so the first item emitted by the + * new Observable will be the result of the function applied to the first item emitted by {@code w0}, the first item emitted by {@code w1}, the first item emitted by {@code w2}, and the first item + * emitted by {@code w3}; the second item emitted by + * the new Observable will be the result of the function applied to the second item emitted by + * each of those Observables; and so forth. + *

+ * The resulting {@code Observable} returned from {@code zip} will invoke {@link Observer#onNext onNext} as many times as the number of {@code onNext} invocations + * of the source Observable that emits the fewest items. + * + * @param o1 + * one source Observable + * @param o2 + * a second source Observable + * @param o3 + * a third source Observable + * @param o4 + * a fourth source Observable + * @param o5 + * a fifth source Observable + * @param o6 + * a sixth source Observable + * @param o7 + * a seventh source Observable + * @param o8 + * an eighth source Observable + * @param zipFunction + * a function that, when applied to an item emitted by each of the source + * Observables, results in an item that will be emitted by the resulting Observable + * @return an Observable that emits the zipped results + */ + public static Observable zip(Observable o1, Observable o2, Observable o3, Observable o4, Observable o5, Observable o6, Observable o7, Observable o8, + Func8 zipFunction) { + return create(OperationZip.zip(o1, o2, o3, o4, o5, o6, o7, o8, zipFunction)); + } + + /** + * Returns an Observable that emits the results of a function of your choosing applied to + * combinations of four items emitted, in sequence, by four other Observables. + *

+ * + *

{@code zip} applies this function in strict sequence, so the first item emitted by the + * new Observable will be the result of the function applied to the first item emitted by {@code w0}, the first item emitted by {@code w1}, the first item emitted by {@code w2}, and the first item + * emitted by {@code w3}; the second item emitted by + * the new Observable will be the result of the function applied to the second item emitted by + * each of those Observables; and so forth. + *

+ * The resulting {@code Observable} returned from {@code zip} will invoke {@link Observer#onNext onNext} as many times as the number of {@code onNext} invocations + * of the source Observable that emits the fewest items. + * + * @param o1 + * one source Observable + * @param o2 + * a second source Observable + * @param o3 + * a third source Observable + * @param o4 + * a fourth source Observable + * @param o5 + * a fifth source Observable + * @param o6 + * a sixth source Observable + * @param o7 + * a seventh source Observable + * @param o8 + * an eighth source Observable + * @param o9 + * a ninth source Observable + * @param zipFunction + * a function that, when applied to an item emitted by each of the source + * Observables, results in an item that will be emitted by the resulting Observable + * @return an Observable that emits the zipped results + */ + public static Observable zip(Observable o1, Observable o2, Observable o3, Observable o4, Observable o5, Observable o6, Observable o7, Observable o8, + Observable o9, Func9 zipFunction) { + return create(OperationZip.zip(o1, o2, o3, o4, o5, o6, o7, o8, o9, zipFunction)); } /** @@ -1034,30 +1233,71 @@ public static Observable zip(Observable w0, Observabl *

* * - * @param w0 + * @param o1 * The first source observable. - * @param w1 + * @param o2 * The second source observable. * @param combineFunction * The aggregation function used to combine the source observable values. * @return An Observable that combines the source Observables with the given combine function */ - public static Observable combineLatest(Observable w0, Observable w1, Func2 combineFunction) { - return create(OperationCombineLatest.combineLatest(w0, w1, combineFunction)); + public static Observable combineLatest(Observable o1, Observable o2, Func2 combineFunction) { + return create(OperationCombineLatest.combineLatest(o1, o2, combineFunction)); + } + + /** + * @see #combineLatest(Observable, Observable, Func2) + */ + public static Observable combineLatest(Observable o1, Observable o2, Observable o3, Func3 combineFunction) { + return create(OperationCombineLatest.combineLatest(o1, o2, o3, combineFunction)); + } + + /** + * @see #combineLatest(Observable, Observable, Func2) + */ + public static Observable combineLatest(Observable o1, Observable o2, Observable o3, Observable o4, + Func4 combineFunction) { + return create(OperationCombineLatest.combineLatest(o1, o2, o3, o4, combineFunction)); } /** * @see #combineLatest(Observable, Observable, Func2) */ - public static Observable combineLatest(Observable w0, Observable w1, Observable w2, Func3 combineFunction) { - return create(OperationCombineLatest.combineLatest(w0, w1, w2, combineFunction)); + public static Observable combineLatest(Observable o1, Observable o2, Observable o3, Observable o4, Observable o5, + Func5 combineFunction) { + return create(OperationCombineLatest.combineLatest(o1, o2, o3, o4, o5, combineFunction)); } /** * @see #combineLatest(Observable, Observable, Func2) */ - public static Observable combineLatest(Observable w0, Observable w1, Observable w2, Observable w3, Func4 combineFunction) { - return create(OperationCombineLatest.combineLatest(w0, w1, w2, w3, combineFunction)); + public static Observable combineLatest(Observable o1, Observable o2, Observable o3, Observable o4, Observable o5, Observable o6, + Func6 combineFunction) { + return create(OperationCombineLatest.combineLatest(o1, o2, o3, o4, o5, o6, combineFunction)); + } + + /** + * @see #combineLatest(Observable, Observable, Func2) + */ + public static Observable combineLatest(Observable o1, Observable o2, Observable o3, Observable o4, Observable o5, Observable o6, Observable o7, + Func7 combineFunction) { + return create(OperationCombineLatest.combineLatest(o1, o2, o3, o4, o5, o6, o7, combineFunction)); + } + + /** + * @see #combineLatest(Observable, Observable, Func2) + */ + public static Observable combineLatest(Observable o1, Observable o2, Observable o3, Observable o4, Observable o5, Observable o6, Observable o7, Observable o8, + Func8 combineFunction) { + return create(OperationCombineLatest.combineLatest(o1, o2, o3, o4, o5, o6, o7, o8, combineFunction)); + } + + /** + * @see #combineLatest(Observable, Observable, Func2) + */ + public static Observable combineLatest(Observable o1, Observable o2, Observable o3, Observable o4, Observable o5, Observable o6, Observable o7, Observable o8, Observable o9, + Func9 combineFunction) { + return create(OperationCombineLatest.combineLatest(o1, o2, o3, o4, o5, o6, o7, o8, o9, combineFunction)); } /** @@ -1075,7 +1315,7 @@ public static Observable combineLatest(Observable w0, * 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. */ - public Observable> buffer(Func0> bufferClosingSelector) { + public Observable> buffer(Func0> bufferClosingSelector) { return create(OperationBuffer.buffer(this, bufferClosingSelector)); } @@ -1097,7 +1337,7 @@ public Observable> buffer(Func0> bufferClosingSelect * @return * An {@link Observable} which produces buffers which are created and emitted when the specified {@link Observable}s publish certain objects. */ - public Observable> buffer(Observable bufferOpenings, Func1> bufferClosingSelector) { + public Observable> buffer(Observable bufferOpenings, Func1> bufferClosingSelector) { return create(OperationBuffer.buffer(this, bufferOpenings, bufferClosingSelector)); } @@ -1263,13 +1503,13 @@ public Observable> buffer(long timespan, long timeshift, TimeUnit unit) public Observable> buffer(long timespan, long timeshift, TimeUnit unit, Scheduler scheduler) { return create(OperationBuffer.buffer(this, timespan, timeshift, unit, scheduler)); } - + /** * 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 {@link Func0} produces a {@link rx.util.Closing} object. The - * {@link Func0} will then be used to create a new Observable to listen for the end of the next window. - * + * Observable produced by the specified {@link Func0} produces a {@link rx.util.Closing} object. The {@link Func0} will then be used to create a new Observable to listen for the end of the next + * window. + * * @param source * The source {@link Observable} which produces values. * @param closingSelector @@ -1277,9 +1517,8 @@ public Observable> buffer(long timespan, long timeshift, TimeUnit unit, * When this {@link Observable} produces a {@link rx.util.Closing} object, the associated window * is emitted and replaced with a new one. * @return - * An {@link Observable} which produces connected non-overlapping windows, which are emitted - * when the current {@link Observable} created with the {@link Func0} argument produces a - * {@link rx.util.Closing} object. + * An {@link Observable} which produces connected non-overlapping windows, which are emitted + * when the current {@link Observable} created with the {@link Func0} argument produces a {@link rx.util.Closing} object. */ public Observable> window(Observable source, Func0> closingSelector) { return create(OperationWindow.window(source, closingSelector)); @@ -1288,9 +1527,9 @@ public Observable> window(Observable source, Func0> window(Observable source, Func0> window(Observable source, Observable windowOpenings, Func1> closingSelector) { return create(OperationWindow.window(source, windowOpenings, closingSelector)); @@ -1312,14 +1550,14 @@ public Observable> window(Observable source, Observable> window(Observable source, int count) { return create(OperationWindow.window(source, count)); @@ -1329,7 +1567,7 @@ public Observable> window(Observable source, int count) { * Creates an Observable which produces windows of collected values. This Observable produces windows every * "skip" values, each containing "count" elements. When the source Observable completes or encounters an error, * the current window is emitted and the event is propagated. - * + * * @param source * The source {@link Observable} which produces values. * @param count @@ -1338,8 +1576,8 @@ public Observable> window(Observable source, int count) { * How many produced values need to be skipped before starting a new window. Note that when "skip" and * "count" are equals that this is the same operation as {@link Observable#window(Observable, int)}. * @return - * An {@link Observable} which produces windows every "skipped" values containing at most - * "count" produced values. + * An {@link Observable} which produces windows every "skipped" values containing at most + * "count" produced values. */ public Observable> window(Observable source, int count, int skip) { return create(OperationWindow.window(source, count, skip)); @@ -1349,7 +1587,7 @@ public Observable> window(Observable source, int count, int ski * Creates an Observable which produces windows of collected values. This Observable produces connected * non-overlapping windows, each of a fixed duration specified by the "timespan" argument. When the source * Observable completes or encounters an error, the current window is emitted and the event is propagated. - * + * * @param source * The source {@link Observable} which produces values. * @param timespan @@ -1358,7 +1596,7 @@ public Observable> window(Observable source, int count, int ski * @param unit * The unit of time which applies to the "timespan" argument. * @return - * An {@link Observable} which produces connected non-overlapping windows with a fixed duration. + * An {@link Observable} which produces connected non-overlapping windows with a fixed duration. */ public Observable> window(Observable source, long timespan, TimeUnit unit) { return create(OperationWindow.window(source, timespan, unit)); @@ -1368,7 +1606,7 @@ public Observable> window(Observable source, long timespan, Tim * Creates an Observable which produces windows of collected values. This Observable produces connected * non-overlapping windows, each of a fixed duration specified by the "timespan" argument. When the source * Observable completes or encounters an error, the current window is emitted and the event is propagated. - * + * * @param source * The source {@link Observable} which produces values. * @param timespan @@ -1379,7 +1617,7 @@ public Observable> window(Observable source, long timespan, Tim * @param scheduler * The {@link Scheduler} to use when determining the end and start of a window. * @return - * An {@link Observable} which produces connected non-overlapping windows with a fixed duration. + * An {@link Observable} which produces connected non-overlapping windows with a fixed duration. */ public Observable> window(Observable source, long timespan, TimeUnit unit, Scheduler scheduler) { return create(OperationWindow.window(source, timespan, unit, scheduler)); @@ -1390,7 +1628,7 @@ public Observable> window(Observable source, long timespan, Tim * non-overlapping windows, each of a fixed duration specified by the "timespan" argument or a maximum size * specified by the "count" argument (which ever is reached first). When the source Observable completes * or encounters an error, the current window is emitted and the event is propagated. - * + * * @param source * The source {@link Observable} which produces values. * @param timespan @@ -1401,8 +1639,8 @@ public Observable> window(Observable source, long timespan, Tim * @param count * The maximum size of each window before it should be emitted. * @return - * An {@link Observable} which produces connected non-overlapping windows which are emitted after - * a fixed duration or when the window has reached maximum capacity (which ever occurs first). + * An {@link Observable} which produces connected non-overlapping windows which are emitted after + * a fixed duration or when the window has reached maximum capacity (which ever occurs first). */ public Observable> window(Observable source, long timespan, TimeUnit unit, int count) { return create(OperationWindow.window(source, timespan, unit, count)); @@ -1413,7 +1651,7 @@ public Observable> window(Observable source, long timespan, Tim * non-overlapping windows, each of a fixed duration specified by the "timespan" argument or a maximum size * specified by the "count" argument (which ever is reached first). When the source Observable completes * or encounters an error, the current window is emitted and the event is propagated. - * + * * @param source * The source {@link Observable} which produces values. * @param timespan @@ -1426,8 +1664,8 @@ public Observable> window(Observable source, long timespan, Tim * @param scheduler * The {@link Scheduler} to use when determining the end and start of a window. * @return - * An {@link Observable} which produces connected non-overlapping windows which are emitted after - * a fixed duration or when the window has reached maximum capacity (which ever occurs first). + * An {@link Observable} which produces connected non-overlapping windows which are emitted after + * a fixed duration or when the window has reached maximum capacity (which ever occurs first). */ public Observable> window(Observable source, long timespan, TimeUnit unit, int count, Scheduler scheduler) { return create(OperationWindow.window(source, timespan, unit, count, scheduler)); @@ -1438,7 +1676,7 @@ public Observable> window(Observable source, long timespan, Tim * periodically, which is determined by the "timeshift" argument. Each window is emitted after a fixed timespan * specified by the "timespan" argument. When the source Observable completes or encounters an error, the * current window is emitted and the event is propagated. - * + * * @param source * The source {@link Observable} which produces values. * @param timespan @@ -1448,8 +1686,8 @@ public Observable> window(Observable source, long timespan, Tim * @param unit * The unit of time which applies to the "timespan" and "timeshift" argument. * @return - * An {@link Observable} which produces new windows periodically, and these are emitted after - * a fixed timespan has elapsed. + * An {@link Observable} which produces new windows periodically, and these are emitted after + * a fixed timespan has elapsed. */ public Observable> window(Observable source, long timespan, long timeshift, TimeUnit unit) { return create(OperationWindow.window(source, timespan, timeshift, unit)); @@ -1460,7 +1698,7 @@ public Observable> window(Observable source, long timespan, lon * periodically, which is determined by the "timeshift" argument. Each window is emitted after a fixed timespan * specified by the "timespan" argument. When the source Observable completes or encounters an error, the * current window is emitted and the event is propagated. - * + * * @param source * The source {@link Observable} which produces values. * @param timespan @@ -1472,13 +1710,13 @@ public Observable> window(Observable source, long timespan, lon * @param scheduler * The {@link Scheduler} to use when determining the end and start of a window. * @return - * An {@link Observable} which produces new windows periodically, and these are emitted after - * a fixed timespan has elapsed. + * An {@link Observable} which produces new windows periodically, and these are emitted after + * a fixed timespan has elapsed. */ public Observable> window(Observable source, long timespan, long timeshift, TimeUnit unit, Scheduler scheduler) { return create(OperationWindow.window(source, timespan, timeshift, unit, scheduler)); } - + /** * Returns an Observable that emits the results of a function of your choosing applied to * combinations of four items emitted, in sequence, by four other Observables. @@ -1494,16 +1732,16 @@ public Observable> window(Observable source, long timespan, lon * * @param ws * An Observable of source Observables - * @param reduceFunction + * @param zipFunction * a function that, when applied to an item emitted by each of the source * Observables, results in an item that will be emitted by the resulting Observable * @return an Observable that emits the zipped results */ - public static Observable zip(Observable> ws, final FuncN reduceFunction) { - return ws.toList().mapMany(new Func1>, Observable>() { + public static Observable zip(Observable> ws, final FuncN zipFunction) { + return ws.toList().mapMany(new Func1>, Observable>() { @Override - public Observable call(List> wsList) { - return create(OperationZip.zip(wsList, reduceFunction)); + public Observable call(List> wsList) { + return create(OperationZip.zip(wsList, zipFunction)); } }); } @@ -1523,13 +1761,13 @@ public Observable call(List> wsList) { * * @param ws * A collection of source Observables - * @param reduceFunction + * @param zipFunction * a function that, when applied to an item emitted by each of the source * Observables, results in an item that will be emitted by the resulting Observable * @return an Observable that emits the zipped results */ - public static Observable zip(Collection> ws, FuncN reduceFunction) { - return create(OperationZip.zip(ws, reduceFunction)); + public static Observable zip(Collection> ws, FuncN zipFunction) { + return create(OperationZip.zip(ws, zipFunction)); } /** @@ -1541,7 +1779,7 @@ public static Observable zip(Collection> ws, FuncN reduc * @return an Observable that emits only those items in the original Observable that the filter * evaluates as {@code true} */ - public Observable filter(Func1 predicate) { + public Observable filter(Func1 predicate) { return create(OperationFilter.filter(this, predicate)); } @@ -1576,7 +1814,7 @@ public Observable finallyDo(Action0 action) { * obtained from this transformation. * @see #mapMany(Func1) */ - public Observable flatMap(Func1> func) { + public Observable flatMap(Func1> func) { return mapMany(func); } @@ -1590,7 +1828,7 @@ public Observable flatMap(Func1> func) { * evaluates as {@code true} * @see #filter(Func1) */ - public Observable where(Func1 predicate) { + public Observable where(Func1 predicate) { return filter(predicate); } @@ -1605,7 +1843,7 @@ public Observable where(Func1 predicate) { * @return an Observable that emits the items from the source Observable, transformed by the * given function */ - public Observable map(Func1 func) { + public Observable map(Func1 func) { return create(OperationMap.map(this, func)); } @@ -1626,7 +1864,7 @@ public Observable map(Func1 func) { * obtained from this transformation. * @see #flatMap(Func1) */ - public Observable mapMany(Func1> func) { + public Observable mapMany(Func1> func) { return create(OperationMap.mapMany(this, func)); } @@ -1685,7 +1923,7 @@ public Observable observeOn(Scheduler scheduler) { */ @SuppressWarnings("unchecked") public Observable dematerialize() { - return create(OperationDematerialize.dematerialize((Observable>) this)); + return create(OperationDematerialize.dematerialize((Observable>) this)); } /** @@ -1712,7 +1950,7 @@ public Observable dematerialize() { * encounters an error * @return the original Observable, with appropriately modified behavior */ - public Observable onErrorResumeNext(final Func1> resumeFunction) { + public Observable onErrorResumeNext(final Func1> resumeFunction) { return create(OperationOnErrorResumeNextViaFunction.onErrorResumeNextViaFunction(this, resumeFunction)); } @@ -1740,7 +1978,7 @@ public Observable onErrorResumeNext(final Func1> res * encounters an error * @return the original Observable, with appropriately modified behavior */ - public Observable onErrorResumeNext(final Observable resumeSequence) { + public Observable onErrorResumeNext(final Observable resumeSequence) { return create(OperationOnErrorResumeNextViaObservable.onErrorResumeNextViaObservable(this, resumeSequence)); } @@ -1770,7 +2008,7 @@ public Observable onErrorResumeNext(final Observable resumeSequence) { * encounters an error * @return the original Observable, with appropriately modified behavior */ - public Observable onExceptionResumeNext(final Observable resumeSequence) { + public Observable onExceptionResumeNext(final Observable resumeSequence) { return create(OperationOnExceptionResumeNextViaObservable.onExceptionResumeNextViaObservable(this, resumeSequence)); } @@ -1797,7 +2035,7 @@ public Observable onExceptionResumeNext(final Observable resumeSequence) { * Observable encounters an error * @return the original Observable with appropriately modified behavior */ - public Observable onErrorReturn(Func1 resumeFunction) { + public Observable onErrorReturn(Func1 resumeFunction) { return create(OperationOnErrorReturn.onErrorReturn(this, resumeFunction)); } @@ -1822,7 +2060,7 @@ public Observable onErrorReturn(Func1 resumeFunction) { * @see MSDN: Observable.Aggregate * @see Wikipedia: Fold (higher-order function) */ - public Observable reduce(Func2 accumulator) { + public Observable reduce(Func2 accumulator) { return create(OperationScan.scan(this, accumulator)).takeLast(1); } @@ -1879,7 +2117,7 @@ public ConnectableObservable publish() { * * @see #reduce(Func2) */ - public Observable aggregate(Func2 accumulator) { + public Observable aggregate(Func2 accumulator) { return reduce(accumulator); } @@ -1906,7 +2144,7 @@ public Observable aggregate(Func2 accumulator) { * @see MSDN: Observable.Aggregate * @see Wikipedia: Fold (higher-order function) */ - public Observable reduce(R initialValue, Func2 accumulator) { + public Observable reduce(R initialValue, Func2 accumulator) { return create(OperationScan.scan(this, initialValue, accumulator)).takeLast(1); } @@ -1917,7 +2155,7 @@ public Observable reduce(R initialValue, Func2 accumulator) { * * @see #reduce(Object, Func2) */ - public Observable aggregate(R initialValue, Func2 accumulator) { + public Observable aggregate(R initialValue, Func2 accumulator) { return reduce(initialValue, accumulator); } @@ -1940,7 +2178,7 @@ public Observable aggregate(R initialValue, Func2 accumulator) { * @return an Observable that emits the results of each call to the accumulator function * @see MSDN: Observable.Scan */ - public Observable scan(Func2 accumulator) { + public Observable scan(Func2 accumulator) { return create(OperationScan.scan(this, accumulator)); } @@ -2001,7 +2239,7 @@ public Observable sample(long period, TimeUnit unit, Scheduler scheduler) { * @return an Observable that emits the results of each call to the accumulator function * @see MSDN: Observable.Scan */ - public Observable scan(R initialValue, Func2 accumulator) { + public Observable scan(R initialValue, Func2 accumulator) { return create(OperationScan.scan(this, initialValue, accumulator)); } @@ -2016,7 +2254,7 @@ public Observable scan(R initialValue, Func2 accumulator) { * @return an Observable that emits true if all items emitted by the source * Observable satisfy the predicate; otherwise, false */ - public Observable all(Func1 predicate) { + public Observable all(Func1 predicate) { return create(OperationAll.all(this, predicate)); } @@ -2069,7 +2307,7 @@ public Observable take(final int num) { * @return an Observable that emits the items from the source Observable so long as each item * satisfies the condition defined by predicate */ - public Observable takeWhile(final Func1 predicate) { + public Observable takeWhile(final Func1 predicate) { return create(OperationTakeWhile.takeWhile(this, predicate)); } @@ -2086,7 +2324,7 @@ public Observable takeWhile(final Func1 predicate) { * @return an Observable that emits items from the source Observable so long as the predicate * continues to return true for each item, then completes */ - public Observable takeWhileWithIndex(final Func2 predicate) { + public Observable takeWhileWithIndex(final Func2 predicate) { return create(OperationTakeWhile.takeWhileWithIndex(this, predicate)); } @@ -2120,7 +2358,7 @@ public Observable takeLast(final int count) { * @return an Observable that emits the items of the source Observable until such time as * other emits its first item */ - public Observable takeUntil(Observable other) { + public Observable takeUntil(Observable other) { return OperationTakeUntil.takeUntil(this, other); } @@ -2172,7 +2410,7 @@ public Observable> toSortedList() { * an Integer that indicates their sort order * @return an Observable that emits the items from the source Observable in sorted order */ - public Observable> toSortedList(Func2 sortFunction) { + public Observable> toSortedList(Func2 sortFunction) { return create(OperationToObservableSortedList.toSortedList(this, sortFunction)); } @@ -2208,7 +2446,7 @@ public Observable startWith(T... values) { * unique key value and emits items representing items from the source Observable that * share that key value */ - public Observable> groupBy(final Func1 keySelector, final Func1 elementSelector) { + public Observable> groupBy(final Func1 keySelector, final Func1 elementSelector) { return create(OperationGroupBy.groupBy(this, keySelector, elementSelector)); } @@ -2226,7 +2464,7 @@ public Observable> groupBy(final Func1 keyS * unique key value and emits items representing items from the source Observable that * share that key value */ - public Observable> groupBy(final Func1 keySelector) { + public Observable> groupBy(final Func1 keySelector) { return create(OperationGroupBy.groupBy(this, keySelector)); } diff --git a/rxjava-core/src/main/java/rx/operators/ChunkedOperation.java b/rxjava-core/src/main/java/rx/operators/ChunkedOperation.java index c7bb5cddbb..5b5eb3344c 100644 --- a/rxjava-core/src/main/java/rx/operators/ChunkedOperation.java +++ b/rxjava-core/src/main/java/rx/operators/ChunkedOperation.java @@ -1,12 +1,12 @@ /** * 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. @@ -15,6 +15,15 @@ */ package rx.operators; +import java.util.ArrayList; +import java.util.List; +import java.util.Queue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + import rx.Observable; import rx.Observer; import rx.Scheduler; @@ -26,22 +35,13 @@ import rx.util.functions.Func0; import rx.util.functions.Func1; -import java.util.ArrayList; -import java.util.List; -import java.util.Queue; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; - /** * The base class for operations that break observables into "chunks". Currently buffers and windows. */ public class ChunkedOperation { /** * This interface defines a way which specifies when to create a new internal {@link rx.operators.ChunkedOperation.Chunk} object. - * + * */ protected interface ChunkCreator { /** @@ -58,16 +58,17 @@ protected interface ChunkCreator { /** * This class represents a single chunk: A sequence of recorded values. - * - * @param The type of objects which this {@link Chunk} can hold. - * The type of object being tracked by the {@link Chunk} + * + * @param + * The type of objects which this {@link Chunk} can hold. + * The type of object being tracked by the {@link Chunk} */ - protected abstract static class Chunk { + protected abstract static class Chunk { protected final List contents = new ArrayList(); /** * Appends a specified value to the {@link Chunk}. - * + * * @param value * The value to append to the {@link Chunk}. */ @@ -77,15 +78,15 @@ public void pushValue(T value) { /** * @return - * The mutable underlying {@link C} which contains all the - * recorded values in this {@link Chunk} object. + * The mutable underlying {@link C} which contains all the + * recorded values in this {@link Chunk} object. */ abstract public C getContents(); /** * @return - * The size of the underlying {@link List} which contains all the - * recorded values in this {@link Chunk} object. + * The size of the underlying {@link List} which contains all the + * recorded values in this {@link Chunk} object. */ public int size() { return contents.size(); @@ -95,19 +96,20 @@ public int size() { /** * This class is an extension on the {@link rx.operators.ChunkedOperation.Chunks} class which only supports one * active (not yet emitted) internal {@link rx.operators.ChunkedOperation.Chunks} object. - * - * @param The type of object all internal {@link rx.operators.ChunkedOperation.Chunks} objects record. - * The type of object being tracked by the {@link Chunk} + * + * @param + * The type of object all internal {@link rx.operators.ChunkedOperation.Chunks} objects record. + * The type of object being tracked by the {@link Chunk} */ - protected static class NonOverlappingChunks extends Chunks { + protected static class NonOverlappingChunks extends Chunks { private final Object lock = new Object(); - public NonOverlappingChunks(Observer observer, Func0 chunkMaker) { + public NonOverlappingChunks(Observer observer, Func0 chunkMaker) { super(observer, chunkMaker); } - public Chunk emitAndReplaceChunk() { + public Chunk emitAndReplaceChunk() { synchronized (lock) { emitChunk(getChunk()); return createChunk(); @@ -126,12 +128,13 @@ public void pushValue(T value) { * This class is an extension on the {@link rx.operators.ChunkedOperation.Chunks} class which actually has no additional * behavior than its super class. Classes extending this class, are expected to support * two or more active (not yet emitted) internal {@link rx.operators.ChunkedOperation.Chunks} objects. - * - * @param The type of object all internal {@link rx.operators.ChunkedOperation.Chunks} objects record. - * The type of object being tracked by the {@link rx.operators.ChunkedOperation.Chunk} + * + * @param + * The type of object all internal {@link rx.operators.ChunkedOperation.Chunks} objects record. + * The type of object being tracked by the {@link rx.operators.ChunkedOperation.Chunk} */ - protected static class OverlappingChunks extends Chunks { - public OverlappingChunks(Observer observer, Func0 chunkMaker) { + protected static class OverlappingChunks extends Chunks { + public OverlappingChunks(Observer observer, Func0 chunkMaker) { super(observer, chunkMaker); } } @@ -141,20 +144,21 @@ public OverlappingChunks(Observer observer, Func0 chunkMaker) { * a has a maximum time to live and a maximum internal capacity. When the chunk has * reached the end of its life, or reached its maximum internal capacity it is * automatically emitted. - * - * @param The type of object all internal {@link rx.operators.ChunkedOperation.Chunk} objects record. - * The type of object being tracked by the {@link Chunk} + * + * @param + * 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 TimeAndSizeBasedChunks extends Chunks { + protected static class TimeAndSizeBasedChunks extends Chunks { - private final ConcurrentMap, Subscription> subscriptions = new ConcurrentHashMap, Subscription>(); + private final ConcurrentMap, Subscription> subscriptions = new ConcurrentHashMap, Subscription>(); private final Scheduler scheduler; private final long maxTime; private final TimeUnit unit; private final int maxSize; - public TimeAndSizeBasedChunks(Observer observer, Func0 chunkMaker, int maxSize, long maxTime, TimeUnit unit, Scheduler scheduler) { + public TimeAndSizeBasedChunks(Observer observer, Func0 chunkMaker, int maxSize, long maxTime, TimeUnit unit, Scheduler scheduler) { super(observer, chunkMaker); this.maxSize = maxSize; this.maxTime = maxTime; @@ -163,8 +167,8 @@ public TimeAndSizeBasedChunks(Observer observer, Func0 chunkMaker, int maxSiz } @Override - public Chunk createChunk() { - final Chunk chunk = super.createChunk(); + public Chunk createChunk() { + final Chunk chunk = super.createChunk(); subscriptions.put(chunk, scheduler.schedule(new Action0() { @Override public void call() { @@ -175,7 +179,7 @@ public void call() { } @Override - public void emitChunk(Chunk chunk) { + public void emitChunk(Chunk chunk) { Subscription subscription = subscriptions.remove(chunk); if (subscription == null) { // Chunk was already emitted. @@ -191,7 +195,7 @@ public void emitChunk(Chunk chunk) { public void pushValue(T value) { super.pushValue(value); - Chunk chunk; + Chunk chunk; while ((chunk = getChunk()) != null) { if (chunk.size() >= maxSize) { emitChunk(chunk); @@ -207,19 +211,20 @@ public void pushValue(T value) { * This class is an extension on the {@link rx.operators.ChunkedOperation.Chunks} class. Every internal chunk has * a has a maximum time to live. When the chunk has reached the end of its life it is * automatically emitted. - * - * @param The type of object all internal {@link rx.operators.ChunkedOperation.Chunk} objects record. - * The type of object being tracked by the {@link Chunk} + * + * @param + * 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 TimeBasedChunks extends OverlappingChunks { + protected static class TimeBasedChunks extends OverlappingChunks { - private final ConcurrentMap, Subscription> subscriptions = new ConcurrentHashMap, Subscription>(); + private final ConcurrentMap, Subscription> subscriptions = new ConcurrentHashMap, Subscription>(); private final Scheduler scheduler; private final long time; private final TimeUnit unit; - public TimeBasedChunks(Observer observer, Func0 chunkMaker, long time, TimeUnit unit, Scheduler scheduler) { + public TimeBasedChunks(Observer observer, Func0 chunkMaker, long time, TimeUnit unit, Scheduler scheduler) { super(observer, chunkMaker); this.time = time; this.unit = unit; @@ -227,8 +232,8 @@ public TimeBasedChunks(Observer observer, Func0 chunkMaker, long time, TimeUn } @Override - public Chunk createChunk() { - final Chunk chunk = super.createChunk(); + public Chunk createChunk() { + final Chunk chunk = super.createChunk(); subscriptions.put(chunk, scheduler.schedule(new Action0() { @Override public void call() { @@ -239,7 +244,7 @@ public void call() { } @Override - public void emitChunk(Chunk chunk) { + public void emitChunk(Chunk chunk) { subscriptions.remove(chunk); super.emitChunk(chunk); } @@ -249,15 +254,16 @@ public void emitChunk(Chunk chunk) { * This class is an extension on the {@link rx.operators.ChunkedOperation.Chunks} class. Every internal chunk has * a fixed maximum capacity. When the chunk has reached its maximum capacity it is * automatically emitted. - * - * @param The type of object all internal {@link rx.operators.ChunkedOperation.Chunk} objects record. - * The type of object being tracked by the {@link Chunk} + * + * @param + * 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 SizeBasedChunks extends Chunks { + protected static class SizeBasedChunks extends Chunks { private final int size; - public SizeBasedChunks(Observer observer, Func0 chunkMaker, int size) { + public SizeBasedChunks(Observer observer, Func0 chunkMaker, int size) { super(observer, chunkMaker); this.size = size; } @@ -266,7 +272,7 @@ public SizeBasedChunks(Observer observer, Func0 chunkMaker, int size) { public void pushValue(T value) { super.pushValue(value); - Chunk chunk; + Chunk chunk; while ((chunk = getChunk()) != null) { if (chunk.size() >= size) { emitChunk(chunk); @@ -280,36 +286,36 @@ public void pushValue(T value) { /** * This class represents an object which contains and manages multiple {@link rx.operators.ChunkedOperation.Chunk} objects. - * - * @param The type of objects which the internal {@link rx.operators.ChunkedOperation.Chunk} objects record. - * The type of object being tracked by the {@link Chunk} + * + * @param + * The type of objects which the internal {@link rx.operators.ChunkedOperation.Chunk} objects record. + * The type of object being tracked by the {@link Chunk} */ - protected static class Chunks { + protected static class Chunks { - private final Queue> chunks = new ConcurrentLinkedQueue>(); - private final Observer observer; + private final Queue> chunks = new ConcurrentLinkedQueue>(); + private final Observer observer; private final Func0> chunkMaker; /** * Constructs a new {@link ChunkedOperation.Chunks} object for the specified {@link rx.Observer}. - * + * * @param observer - * The {@link rx.Observer} to which this object will emit its internal - * {@link rx.operators.ChunkedOperation.Chunk} objects to when requested. + * The {@link rx.Observer} to which this object will emit its internal {@link rx.operators.ChunkedOperation.Chunk} objects to when requested. */ - public Chunks(Observer observer, Func0 chunkMaker) { + public Chunks(Observer observer, Func0 chunkMaker) { this.observer = observer; this.chunkMaker = chunkMaker; } /** * This method will instantiate a new {@link rx.operators.ChunkedOperation.Chunk} object and register it internally. - * + * * @return - * The constructed empty {@link rx.operators.ChunkedOperation.Chunk} object. + * The constructed empty {@link rx.operators.ChunkedOperation.Chunk} object. */ - public Chunk createChunk() { - Chunk chunk = chunkMaker.call(); + public Chunk createChunk() { + Chunk chunk = chunkMaker.call(); chunks.add(chunk); return chunk; } @@ -318,7 +324,7 @@ public Chunk createChunk() { * This method emits all not yet emitted {@link rx.operators.ChunkedOperation.Chunk} objects. */ public void emitAllChunks() { - Chunk chunk; + Chunk chunk; while ((chunk = chunks.poll()) != null) { observer.onNext(chunk.getContents()); } @@ -326,11 +332,11 @@ public void emitAllChunks() { /** * This method emits the specified {@link rx.operators.ChunkedOperation.Chunk} object. - * + * * @param chunk * The {@link rx.operators.ChunkedOperation.Chunk} to emit. */ - public void emitChunk(Chunk chunk) { + public void emitChunk(Chunk chunk) { if (!chunks.remove(chunk)) { // Concurrency issue: Chunk is already emitted! return; @@ -340,21 +346,21 @@ public void emitChunk(Chunk chunk) { /** * @return - * The oldest (in case there are multiple) {@link rx.operators.ChunkedOperation.Chunk} object. + * The oldest (in case there are multiple) {@link rx.operators.ChunkedOperation.Chunk} object. */ - public Chunk getChunk() { + public Chunk getChunk() { return chunks.peek(); } /** * This method pushes a value to all not yet emitted {@link rx.operators.ChunkedOperation.Chunk} objects. - * + * * @param value * The value to push to all not yet emitted {@link rx.operators.ChunkedOperation.Chunk} objects. */ public void pushValue(T value) { - List> copy = new ArrayList>(chunks); - for (Chunk chunk : copy) { + List> copy = new ArrayList>(chunks); + for (Chunk chunk : copy) { chunk.pushValue(value); } } @@ -362,21 +368,22 @@ public void pushValue(T value) { /** * This {@link rx.operators.ChunkedOperation.ChunkObserver} object can be constructed using a {@link rx.operators.ChunkedOperation.Chunks} object, - * a {@link rx.Observer} object, and a {@link rx.operators.ChunkedOperation.ChunkCreator} object. The {@link rx.operators.ChunkedOperation.ChunkCreator} - * will manage the creation, and in some rare cases emission of internal {@link rx.operators.ChunkedOperation.Chunk} objects - * in the specified {@link rx.operators.ChunkedOperation.Chunks} object. Under normal circumstances the {@link rx.operators.ChunkedOperation.Chunks} - * object specifies when a created {@link rx.operators.ChunkedOperation.Chunk} is emitted. - * - * @param The type of object all internal {@link rx.operators.ChunkedOperation.Chunk} objects record. - * The type of object being tracked by the {@link Chunk} + * a {@link rx.Observer} object, and a {@link rx.operators.ChunkedOperation.ChunkCreator} object. The {@link rx.operators.ChunkedOperation.ChunkCreator} will manage the creation, and in some rare + * cases emission of internal {@link rx.operators.ChunkedOperation.Chunk} objects + * in the specified {@link rx.operators.ChunkedOperation.Chunks} object. Under normal circumstances the {@link rx.operators.ChunkedOperation.Chunks} object specifies when a created + * {@link rx.operators.ChunkedOperation.Chunk} is emitted. + * + * @param + * 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 ChunkObserver implements Observer { + protected static class ChunkObserver implements Observer { - private final Chunks chunks; - private final Observer observer; + private final Chunks chunks; + private final Observer observer; private final ChunkCreator creator; - public ChunkObserver(Chunks chunks, Observer observer, ChunkCreator creator) { + public ChunkObserver(Chunks chunks, Observer observer, ChunkCreator creator) { this.observer = observer; this.creator = creator; this.chunks = chunks; @@ -407,11 +414,12 @@ public void onNext(T args) { * This {@link rx.operators.ChunkedOperation.ChunkCreator} creates a new {@link rx.operators.ChunkedOperation.Chunk} when it is initialized, but * provides no additional functionality. This class should primarily be used when the * internal {@link rx.operators.ChunkedOperation.Chunk} is closed externally. - * - * @param The type of object all internal {@link rx.operators.ChunkedOperation.Chunk} objects record. - * The type of object being tracked by the {@link Chunk} + * + * @param + * 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 SingleChunkCreator implements ChunkCreator { + protected static class SingleChunkCreator implements ChunkCreator { public SingleChunkCreator(Chunks chunks) { chunks.createChunk(); @@ -432,17 +440,18 @@ public void stop() { * This {@link rx.operators.ChunkedOperation.ChunkCreator} creates a new {@link rx.operators.ChunkedOperation.Chunk} whenever it receives an * object from the provided {@link rx.Observable} created with the * chunkClosingSelector {@link rx.util.functions.Func0}. - * - * @param The type of object all internal {@link rx.operators.ChunkedOperation.Chunk} objects record. - * The type of object being tracked by the {@link Chunk} + * + * @param + * 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; @@ -451,7 +460,7 @@ public ObservableBasedSingleChunkCreator(NonOverlappingChunks chunks, Func0 } private void listenForChunkEnd() { - Observable closingObservable = chunkClosingSelector.call(); + Observable closingObservable = chunkClosingSelector.call(); closingObservable.subscribe(new Action1() { @Override public void call(Closing closing) { @@ -474,23 +483,24 @@ public void stop() { /** * This {@link rx.operators.ChunkedOperation.ChunkCreator} creates a new {@link rx.operators.ChunkedOperation.Chunk} whenever it receives - * an object from the provided chunkOpenings {@link rx.Observable}, and closes the corresponding - * {@link rx.operators.ChunkedOperation.Chunk} object when it receives an object from the provided {@link rx.Observable} created + * an object from the provided chunkOpenings {@link rx.Observable}, and closes the corresponding {@link rx.operators.ChunkedOperation.Chunk} object when it receives an object from the provided + * {@link rx.Observable} created * with the chunkClosingSelector {@link rx.util.functions.Func1}. - * - * @param The type of object all internal {@link rx.operators.ChunkedOperation.Chunk} objects record. - * The type of object being tracked by the {@link Chunk} + * + * @param + * 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) { + public ObservableBasedMultiChunkCreator(final OverlappingChunks chunks, Observable openings, final Func1> chunkClosingSelector) { subscription.wrap(openings.subscribe(new Action1() { @Override public void call(Opening opening) { final Chunk chunk = chunks.createChunk(); - Observable closingObservable = chunkClosingSelector.call(opening); + Observable closingObservable = chunkClosingSelector.call(opening); closingObservable.subscribe(new Action1() { @Override @@ -516,15 +526,16 @@ public void stop() { /** * This {@link rx.operators.ChunkedOperation.ChunkCreator} creates a new {@link rx.operators.ChunkedOperation.Chunk} every time after a fixed * period of time has elapsed. - * - * @param The type of object all internal {@link rx.operators.ChunkedOperation.Chunk} objects record. - * The type of object being tracked by the {@link Chunk} + * + * @param + * 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 TimeBasedChunkCreator implements ChunkCreator { + protected static class TimeBasedChunkCreator implements ChunkCreator { private final SafeObservableSubscription subscription = new SafeObservableSubscription(); - public TimeBasedChunkCreator(final NonOverlappingChunks chunks, long time, TimeUnit unit, Scheduler scheduler) { + public TimeBasedChunkCreator(final NonOverlappingChunks chunks, long time, TimeUnit unit, Scheduler scheduler) { this.subscription.wrap(scheduler.schedulePeriodically(new Action0() { @Override public void call() { @@ -533,7 +544,7 @@ public void call() { }, 0, time, unit)); } - public TimeBasedChunkCreator(final OverlappingChunks chunks, long time, TimeUnit unit, Scheduler scheduler) { + public TimeBasedChunkCreator(final OverlappingChunks chunks, long time, TimeUnit unit, Scheduler scheduler) { this.subscription.wrap(scheduler.schedulePeriodically(new Action0() { @Override public void call() { @@ -556,17 +567,18 @@ public void stop() { /** * This {@link rx.operators.ChunkedOperation.ChunkCreator} creates a new {@link rx.operators.ChunkedOperation.Chunk} every time after it has * seen a certain amount of elements. - * - * @param The type of object all internal {@link rx.operators.ChunkedOperation.Chunk} objects record. - * The type of object being tracked by the {@link Chunk} + * + * @param + * 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 SkippingChunkCreator implements ChunkCreator { + protected static class SkippingChunkCreator implements ChunkCreator { private final AtomicInteger skipped = new AtomicInteger(1); - private final Chunks chunks; + private final Chunks chunks; private final int skip; - public SkippingChunkCreator(Chunks chunks, int skip) { + public SkippingChunkCreator(Chunks chunks, int skip) { this.chunks = chunks; this.skip = skip; } diff --git a/rxjava-core/src/main/java/rx/operators/OperationBuffer.java b/rxjava-core/src/main/java/rx/operators/OperationBuffer.java index 2323f6478e..c692ed6cee 100644 --- a/rxjava-core/src/main/java/rx/operators/OperationBuffer.java +++ b/rxjava-core/src/main/java/rx/operators/OperationBuffer.java @@ -6,7 +6,7 @@ * 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. @@ -25,6 +25,7 @@ import org.mockito.Mockito; import rx.Observable; +import rx.Observable.OnSubscribeFunc; import rx.Observer; import rx.Scheduler; import rx.Subscription; @@ -39,10 +40,9 @@ import rx.util.functions.Func0; import rx.util.functions.Func1; - public final class OperationBuffer extends ChunkedOperation { - public static final Func0 BUFFER_MAKER = new Func0() { + private static final Func0 BUFFER_MAKER = new Func0() { @Override public Object call() { return new Buffer(); @@ -51,29 +51,29 @@ public Object 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} value. The buffer is then + * 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} + * 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 event is propagated + * 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 + * event is propagated * to all subscribed {@link Observer}s.

- * + * *

Note that this operation only produces non-overlapping chunks. At all times there is * exactly one buffer actively storing values.

- * + * * @param source * 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 + * 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. * @return - * the {@link Func1} object representing the specified buffer operation. + * the {@link Func1} object representing the specified buffer operation. */ - public static Func1>, Subscription> buffer(final Observable source, final Func0> bufferClosingSelector) { - return new Func1>, Subscription>() { + public static OnSubscribeFunc> buffer(final Observable source, final Func0> bufferClosingSelector) { + return new OnSubscribeFunc>() { + @Override - public Subscription call(final Observer> observer) { + public Subscription onSubscribe(Observer> observer) { NonOverlappingChunks> buffers = new NonOverlappingChunks>(observer, BUFFER_MAKER); ChunkCreator creator = new ObservableBasedSingleChunkCreator>(buffers, bufferClosingSelector); return source.subscribe(new ChunkObserver>(buffers, observer, creator)); @@ -85,33 +85,31 @@ public Subscription call(final Observer> observer) { *

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 the currently active chunks. Initially * there are no chunks active.

- * + * *

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 + * 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 * 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.

- * + * *

Note that when using this operation multiple overlapping chunks * could be active at any one point.

- * + * * @param source * The {@link Observable} which produces values. * @param bufferOpenings * An {@link Observable} which when it produces a {@link rx.util.Opening} value will * 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 + * 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. * @return - * the {@link Func1} object representing the specified buffer operation. + * the {@link Func1} object representing the specified buffer operation. */ - public static Func1>, Subscription> buffer(final Observable source, final Observable bufferOpenings, final Func1> bufferClosingSelector) { - return new Func1>, Subscription>() { + public static OnSubscribeFunc> buffer(final Observable source, final Observable bufferOpenings, final Func1> bufferClosingSelector) { + return new OnSubscribeFunc>() { @Override - public Subscription call(final Observer> observer) { + public Subscription onSubscribe(final Observer> observer) { OverlappingChunks> buffers = new OverlappingChunks>(observer, BUFFER_MAKER); ChunkCreator creator = new ObservableBasedMultiChunkCreator>(buffers, bufferOpenings, bufferClosingSelector); return source.subscribe(new ChunkObserver>(buffers, observer, creator)); @@ -125,18 +123,18 @@ public Subscription call(final Observer> observer) { * a specified number of elements. The buffer is then emitted, and a new buffer is created to replace it. * When the source {@link Observable} completes or produces an error, the current buffer is emitted, and * the event is propagated to all subscribed {@link Observer}s.

- * + * *

Note that this operation only produces non-overlapping chunks. At all times there is * exactly one buffer actively storing values.

- * + * * @param source * The {@link Observable} which produces values. * @param count * The number of elements a buffer should have before being emitted and replaced. * @return - * the {@link Func1} object representing the specified buffer operation. + * the {@link Func1} object representing the specified buffer operation. */ - public static Func1>, Subscription> buffer(Observable source, int count) { + public static OnSubscribeFunc> buffer(Observable source, int count) { return buffer(source, count, count); } @@ -146,10 +144,10 @@ public static Func1>, Subscription> buffer(Observable so * contains a specified number of elements. The buffer is then emitted. Chunks are created after a certain * amount of values have been received. When the source {@link Observable} completes or produces an error, the * currently active chunks are emitted, and the event is propagated to all subscribed {@link Observer}s.

- * + * *

Note that this operation can produce non-connected, connected non-overlapping, or overlapping * chunks depending on the input parameters.

- * + * * @param source * The {@link Observable} which produces values. * @param count @@ -161,12 +159,12 @@ public static Func1>, Subscription> buffer(Observable so * > "count" non-overlapping chunks will be created and some values will not be pushed * into a buffer at all! * @return - * the {@link Func1} object representing the specified buffer operation. + * the {@link Func1} object representing the specified buffer operation. */ - public static Func1>, Subscription> buffer(final Observable source, final int count, final int skip) { - return new Func1>, Subscription>() { + public static OnSubscribeFunc> buffer(final Observable source, final int count, final int skip) { + return new OnSubscribeFunc>() { @Override - public Subscription call(final Observer> observer) { + public Subscription onSubscribe(final Observer> observer) { Chunks> chunks = new SizeBasedChunks>(observer, BUFFER_MAKER, count); ChunkCreator creator = new SkippingChunkCreator>(chunks, skip); return source.subscribe(new ChunkObserver>(chunks, observer, creator)); @@ -180,10 +178,10 @@ public Subscription call(final Observer> observer) { * is emitted and replaced with a new buffer. How often this is done depends on the specified timespan. * When the source {@link Observable} completes or produces an error, the current buffer is emitted, and * the event is propagated to all subscribed {@link Observer}s.

- * + * *

Note that this operation only produces non-overlapping chunks. At all times there is * exactly one buffer actively storing values.

- * + * * @param source * The {@link Observable} which produces values. * @param timespan @@ -191,9 +189,9 @@ public Subscription call(final Observer> observer) { * @param unit * The {@link TimeUnit} defining the unit of time for the timespan. * @return - * the {@link Func1} object representing the specified buffer operation. + * the {@link Func1} object representing the specified buffer operation. */ - public static Func1>, Subscription> buffer(Observable source, long timespan, TimeUnit unit) { + public static OnSubscribeFunc> buffer(Observable source, long timespan, TimeUnit unit) { return buffer(source, timespan, unit, Schedulers.threadPoolForComputation()); } @@ -203,10 +201,10 @@ public static Func1>, Subscription> buffer(Observable so * is emitted and replaced with a new buffer. How often this is done depends on the specified timespan. * When the source {@link Observable} completes or produces an error, the current buffer is emitted, and * the event is propagated to all subscribed {@link Observer}s.

- * + * *

Note that this operation only produces non-overlapping chunks. At all times there is * exactly one buffer actively storing values.

- * + * * @param source * The {@link Observable} which produces values. * @param timespan @@ -216,12 +214,12 @@ public static Func1>, Subscription> buffer(Observable so * @param scheduler * The {@link Scheduler} to use for timing chunks. * @return - * the {@link Func1} object representing the specified buffer operation. + * the {@link Func1} object representing the specified buffer operation. */ - public static Func1>, Subscription> buffer(final Observable source, final long timespan, final TimeUnit unit, final Scheduler scheduler) { - return new Func1>, Subscription>() { + public static OnSubscribeFunc> buffer(final Observable source, final long timespan, final TimeUnit unit, final Scheduler scheduler) { + return new OnSubscribeFunc>() { @Override - public Subscription call(final Observer> observer) { + public Subscription onSubscribe(final Observer> observer) { NonOverlappingChunks> buffers = new NonOverlappingChunks>(observer, BUFFER_MAKER); ChunkCreator creator = new TimeBasedChunkCreator>(buffers, timespan, unit, scheduler); return source.subscribe(new ChunkObserver>(buffers, observer, creator)); @@ -236,10 +234,10 @@ public Subscription call(final Observer> observer) { * Additionally the buffer is automatically emitted once it reaches a specified number of elements. * When the source {@link Observable} completes or produces an error, the current buffer is emitted, and * the event is propagated to all subscribed {@link Observer}s.

- * + * *

Note that this operation only produces non-overlapping chunks. At all times there is * exactly one buffer actively storing values.

- * + * * @param source * The {@link Observable} which produces values. * @param timespan @@ -249,9 +247,9 @@ public Subscription call(final Observer> observer) { * @param count * The maximum size of the buffer. Once a buffer reaches this size, it is emitted. * @return - * the {@link Func1} object representing the specified buffer operation. + * the {@link Func1} object representing the specified buffer operation. */ - public static Func1>, Subscription> buffer(Observable source, long timespan, TimeUnit unit, int count) { + public static OnSubscribeFunc> buffer(Observable source, long timespan, TimeUnit unit, int count) { return buffer(source, timespan, unit, count, Schedulers.threadPoolForComputation()); } @@ -262,10 +260,10 @@ public static Func1>, Subscription> buffer(Observable so * Additionally the buffer is automatically emitted once it reaches a specified number of elements. * When the source {@link Observable} completes or produces an error, the current buffer is emitted, and * the event is propagated to all subscribed {@link Observer}s.

- * + * *

Note that this operation only produces non-overlapping chunks. At all times there is * exactly one buffer actively storing values.

- * + * * @param source * The {@link Observable} which produces values. * @param timespan @@ -277,12 +275,12 @@ public static Func1>, Subscription> buffer(Observable so * @param scheduler * The {@link Scheduler} to use for timing chunks. * @return - * the {@link Func1} object representing the specified buffer operation. + * the {@link Func1} object representing the specified buffer operation. */ - public static Func1>, Subscription> buffer(final Observable source, final long timespan, final TimeUnit unit, final int count, final Scheduler scheduler) { - return new Func1>, Subscription>() { + public static OnSubscribeFunc> buffer(final Observable source, final long timespan, final TimeUnit unit, final int count, final Scheduler scheduler) { + return new OnSubscribeFunc>() { @Override - public Subscription call(final Observer> observer) { + public Subscription onSubscribe(final Observer> observer) { Chunks> chunks = new TimeAndSizeBasedChunks>(observer, BUFFER_MAKER, count, timespan, unit, scheduler); ChunkCreator creator = new SingleChunkCreator>(chunks); return source.subscribe(new ChunkObserver>(chunks, observer, creator)); @@ -297,10 +295,10 @@ public Subscription call(final Observer> observer) { * The creation of chunks is also periodical. How often this is done depends on the specified timeshift. * When the source {@link Observable} completes or produces an error, the current buffer is emitted, and * the event is propagated to all subscribed {@link Observer}s.

- * + * *

Note that this operation can produce non-connected, or overlapping chunks depending * on the input parameters.

- * + * * @param source * The {@link Observable} which produces values. * @param timespan @@ -310,9 +308,9 @@ public Subscription call(final Observer> observer) { * @param unit * The {@link TimeUnit} defining the unit of time for the timespan. * @return - * the {@link Func1} object representing the specified buffer operation. + * the {@link Func1} object representing the specified buffer operation. */ - public static Func1>, Subscription> buffer(Observable source, long timespan, long timeshift, TimeUnit unit) { + public static OnSubscribeFunc> buffer(Observable source, long timespan, long timeshift, TimeUnit unit) { return buffer(source, timespan, timeshift, unit, Schedulers.threadPoolForComputation()); } @@ -323,10 +321,10 @@ public static Func1>, Subscription> buffer(Observable so * The creation of chunks is also periodical. How often this is done depends on the specified timeshift. * When the source {@link Observable} completes or produces an error, the current buffer is emitted, and * the event is propagated to all subscribed {@link Observer}s.

- * + * *

Note that this operation can produce non-connected, or overlapping chunks depending * on the input parameters.

- * + * * @param source * The {@link Observable} which produces values. * @param timespan @@ -338,12 +336,12 @@ public static Func1>, Subscription> buffer(Observable so * @param scheduler * The {@link Scheduler} to use for timing chunks. * @return - * the {@link Func1} object representing the specified buffer operation. + * the {@link Func1} object representing the specified buffer operation. */ - public static Func1>, Subscription> buffer(final Observable source, final long timespan, final long timeshift, final TimeUnit unit, final Scheduler scheduler) { - return new Func1>, Subscription>() { + public static OnSubscribeFunc> buffer(final Observable source, final long timespan, final long timeshift, final TimeUnit unit, final Scheduler scheduler) { + return new OnSubscribeFunc>() { @Override - public Subscription call(final Observer> observer) { + public Subscription onSubscribe(final Observer> observer) { OverlappingChunks> buffers = new TimeBasedChunks>(observer, BUFFER_MAKER, timespan, unit, scheduler); ChunkCreator creator = new TimeBasedChunkCreator>(buffers, timeshift, unit, scheduler); return source.subscribe(new ChunkObserver>(buffers, observer, creator)); @@ -354,13 +352,14 @@ public Subscription call(final Observer> observer) { /** * This class represents a single buffer: A sequence of recorded values. * - * @param The type of objects which this {@link Buffer} can hold. + * @param + * The type of objects which this {@link Buffer} can hold. */ protected static class Buffer extends Chunk> { /** * @return - * The mutable underlying {@link List} which contains all the - * recorded values in this {@link Buffer} object. + * The mutable underlying {@link List} which contains all the + * recorded values in this {@link Buffer} object. */ @Override public List getContents() { @@ -382,9 +381,9 @@ public void before() { @Test public void testComplete() { - Observable source = Observable.create(new Func1, Subscription>() { + Observable source = Observable.create(new OnSubscribeFunc() { @Override - public Subscription call(Observer observer) { + public Subscription onSubscribe(Observer observer) { observer.onCompleted(); return Subscriptions.empty(); } @@ -400,9 +399,9 @@ public Subscription call(Observer observer) { @Test public void testSkipAndCountOverlappingBuffers() { - Observable source = Observable.create(new Func1, Subscription>() { + Observable source = Observable.create(new OnSubscribeFunc() { @Override - public Subscription call(Observer observer) { + public Subscription onSubscribe(Observer observer) { observer.onNext("one"); observer.onNext("two"); observer.onNext("three"); @@ -426,9 +425,9 @@ public Subscription call(Observer observer) { @Test public void testSkipAndCountGaplessBuffers() { - Observable source = Observable.create(new Func1, Subscription>() { + Observable source = Observable.create(new OnSubscribeFunc() { @Override - public Subscription call(Observer observer) { + public Subscription onSubscribe(Observer observer) { observer.onNext("one"); observer.onNext("two"); observer.onNext("three"); @@ -452,9 +451,9 @@ public Subscription call(Observer observer) { @Test public void testSkipAndCountBuffersWithGaps() { - Observable source = Observable.create(new Func1, Subscription>() { + Observable source = Observable.create(new OnSubscribeFunc() { @Override - public Subscription call(Observer observer) { + public Subscription onSubscribe(Observer observer) { observer.onNext("one"); observer.onNext("two"); observer.onNext("three"); @@ -478,9 +477,9 @@ public Subscription call(Observer observer) { @Test public void testTimedAndCount() { - Observable source = Observable.create(new Func1, Subscription>() { + Observable source = Observable.create(new OnSubscribeFunc() { @Override - public Subscription call(Observer observer) { + public Subscription onSubscribe(Observer observer) { push(observer, "one", 10); push(observer, "two", 90); push(observer, "three", 110); @@ -510,9 +509,9 @@ public Subscription call(Observer observer) { @Test public void testTimed() { - Observable source = Observable.create(new Func1, Subscription>() { + Observable source = Observable.create(new OnSubscribeFunc() { @Override - public Subscription call(Observer observer) { + public Subscription onSubscribe(Observer observer) { push(observer, "one", 98); push(observer, "two", 99); push(observer, "three", 100); @@ -539,9 +538,9 @@ public Subscription call(Observer observer) { @Test public void testObservableBasedOpenerAndCloser() { - Observable source = Observable.create(new Func1, Subscription>() { + Observable source = Observable.create(new OnSubscribeFunc() { @Override - public Subscription call(Observer observer) { + public Subscription onSubscribe(Observer observer) { push(observer, "one", 10); push(observer, "two", 60); push(observer, "three", 110); @@ -552,9 +551,9 @@ public Subscription call(Observer observer) { } }); - Observable openings = Observable.create(new Func1, Subscription>() { + Observable openings = Observable.create(new OnSubscribeFunc() { @Override - public Subscription call(Observer observer) { + public Subscription onSubscribe(Observer observer) { push(observer, Openings.create(), 50); push(observer, Openings.create(), 200); complete(observer, 250); @@ -565,9 +564,9 @@ public Subscription call(Observer observer) { Func1> closer = new Func1>() { @Override public Observable call(Opening opening) { - return Observable.create(new Func1, Subscription>() { + return Observable.create(new OnSubscribeFunc() { @Override - public Subscription call(Observer observer) { + public Subscription onSubscribe(Observer observer) { push(observer, Closings.create(), 100); complete(observer, 101); return Subscriptions.empty(); @@ -590,9 +589,9 @@ public Subscription call(Observer observer) { @Test public void testObservableBasedCloser() { - Observable source = Observable.create(new Func1, Subscription>() { + Observable source = Observable.create(new OnSubscribeFunc() { @Override - public Subscription call(Observer observer) { + public Subscription onSubscribe(Observer observer) { push(observer, "one", 10); push(observer, "two", 60); push(observer, "three", 110); @@ -606,9 +605,9 @@ public Subscription call(Observer observer) { Func0> closer = new Func0>() { @Override public Observable call() { - return Observable.create(new Func1, Subscription>() { + return Observable.create(new OnSubscribeFunc() { @Override - public Subscription call(Observer observer) { + public Subscription onSubscribe(Observer observer) { push(observer, Closings.create(), 100); complete(observer, 101); return Subscriptions.empty(); diff --git a/rxjava-core/src/main/java/rx/operators/OperationWindow.java b/rxjava-core/src/main/java/rx/operators/OperationWindow.java index ce603de8cf..0278fbb4d8 100644 --- a/rxjava-core/src/main/java/rx/operators/OperationWindow.java +++ b/rxjava-core/src/main/java/rx/operators/OperationWindow.java @@ -1,12 +1,12 @@ /** * 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. @@ -15,9 +15,17 @@ */ package rx.operators; +import static org.junit.Assert.*; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; + import org.junit.Before; import org.junit.Test; + import rx.Observable; +import rx.Observable.OnSubscribeFunc; import rx.Observer; import rx.Scheduler; import rx.Subscription; @@ -28,15 +36,10 @@ import rx.util.Closings; import rx.util.Opening; import rx.util.Openings; -import rx.util.functions.*; - -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.TimeUnit; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; - +import rx.util.functions.Action0; +import rx.util.functions.Action1; +import rx.util.functions.Func0; +import rx.util.functions.Func1; public final class OperationWindow extends ChunkedOperation { @@ -49,33 +52,33 @@ public Object 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 + * 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 * 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 + * 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 * to all subscribed {@link rx.Observer}s.

- * + * *

Note that this operation only produces non-overlapping windows. At all times there is * exactly one window actively storing values.

- * + * * @param source * 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 + * 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. * @return - * the {@link rx.util.functions.Func1} object representing the specified window operation. + * the {@link rx.util.functions.Func1} object representing the specified window operation. */ - public static Func1>, Subscription> window(final Observable source, final Func0> windowClosingSelector) { - return new Func1>, Subscription>() { + public static OnSubscribeFunc> window(final Observable source, final Func0> windowClosingSelector) { + return new OnSubscribeFunc>() { @Override - public Subscription call(final Observer> observer) { + public Subscription onSubscribe(final Observer> observer) { NonOverlappingChunks> windows = new NonOverlappingChunks>(observer, WINDOW_MAKER); ChunkCreator creator = new ObservableBasedSingleChunkCreator>(windows, windowClosingSelector); return source.subscribe(new ChunkObserver>(windows, observer, creator)); } + }; } @@ -83,33 +86,31 @@ public Subscription call(final Observer> observer) { *

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 the currently active window. Initially * there are no windows active.

- * + * *

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 + * 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 * 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.

- * + * *

Note that when using this operation multiple overlapping windows * could be active at any one point.

- * + * * @param source * The {@link rx.Observable} which produces values. * @param windowOpenings * An {@link rx.Observable} which when it produces a {@link rx.util.Opening} value will * 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 + * 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. * @return - * the {@link rx.util.functions.Func1} object representing the specified window operation. + * the {@link rx.util.functions.Func1} object representing the specified window operation. */ - public static Func1>, Subscription> window(final Observable source, final Observable windowOpenings, final Func1> windowClosingSelector) { - return new Func1>, Subscription>() { + public static OnSubscribeFunc> window(final Observable source, final Observable windowOpenings, final Func1> windowClosingSelector) { + return new OnSubscribeFunc>() { @Override - public Subscription call(final Observer> observer) { + public Subscription onSubscribe(final Observer> observer) { OverlappingChunks> windows = new OverlappingChunks>(observer, WINDOW_MAKER); ChunkCreator creator = new ObservableBasedMultiChunkCreator>(windows, windowOpenings, windowClosingSelector); return source.subscribe(new ChunkObserver>(windows, observer, creator)); @@ -123,18 +124,18 @@ public Subscription call(final Observer> observer) { * a specified number of elements. The window is then emitted, and a new window is created to replace it. * When the source {@link rx.Observable} completes or produces an error, the current window is emitted, and * the event is propagated to all subscribed {@link rx.Observer}s.

- * + * *

Note that this operation only produces non-overlapping windows. At all times there is * exactly one window actively storing values.

- * + * * @param source * The {@link rx.Observable} which produces values. * @param count * The number of elements a window should have before being emitted and replaced. * @return - * the {@link rx.util.functions.Func1} object representing the specified window operation. + * the {@link rx.util.functions.Func1} object representing the specified window operation. */ - public static Func1>, Subscription> window(Observable source, int count) { + public static OnSubscribeFunc> window(Observable source, int count) { return window(source, count, count); } @@ -144,10 +145,10 @@ public static Func1>, Subscription> window(Observable * contains a specified number of elements. The window is then emitted. windows are created after a certain * amount of values have been received. When the source {@link rx.Observable} completes or produces an error, the * currently active windows are emitted, and the event is propagated to all subscribed {@link rx.Observer}s.

- * + * *

Note that this operation can produce non-connected, connected non-overlapping, or overlapping * windows depending on the input parameters.

- * + * * @param source * The {@link rx.Observable} which produces values. * @param count @@ -159,12 +160,12 @@ public static Func1>, Subscription> window(Observable * > "count" non-overlapping windows will be created and some values will not be pushed * into a window at all! * @return - * the {@link rx.util.functions.Func1} object representing the specified window operation. + * the {@link rx.util.functions.Func1} object representing the specified window operation. */ - public static Func1>, Subscription> window(final Observable source, final int count, final int skip) { - return new Func1>, Subscription>() { + public static OnSubscribeFunc> window(final Observable source, final int count, final int skip) { + return new OnSubscribeFunc>() { @Override - public Subscription call(final Observer> observer) { + public Subscription onSubscribe(final Observer> observer) { Chunks> chunks = new SizeBasedChunks>(observer, WINDOW_MAKER, count); ChunkCreator creator = new SkippingChunkCreator>(chunks, skip); return source.subscribe(new ChunkObserver>(chunks, observer, creator)); @@ -178,10 +179,10 @@ public Subscription call(final Observer> observer) { * is emitted and replaced with a new window. How often this is done depends on the specified timespan. * When the source {@link rx.Observable} completes or produces an error, the current window is emitted, and * the event is propagated to all subscribed {@link rx.Observer}s.

- * + * *

Note that this operation only produces non-overlapping windows. At all times there is * exactly one window actively storing values.

- * + * * @param source * The {@link rx.Observable} which produces values. * @param timespan @@ -189,9 +190,9 @@ public Subscription call(final Observer> observer) { * @param unit * The {@link java.util.concurrent.TimeUnit} defining the unit of time for the timespan. * @return - * the {@link rx.util.functions.Func1} object representing the specified window operation. + * the {@link rx.util.functions.Func1} object representing the specified window operation. */ - public static Func1>, Subscription> window(Observable source, long timespan, TimeUnit unit) { + public static OnSubscribeFunc> window(Observable source, long timespan, TimeUnit unit) { return window(source, timespan, unit, Schedulers.threadPoolForComputation()); } @@ -201,10 +202,10 @@ public static Func1>, Subscription> window(Observable * is emitted and replaced with a new window. How often this is done depends on the specified timespan. * When the source {@link rx.Observable} completes or produces an error, the current window is emitted, and * the event is propagated to all subscribed {@link rx.Observer}s.

- * + * *

Note that this operation only produces non-overlapping windows. At all times there is * exactly one window actively storing values.

- * + * * @param source * The {@link rx.Observable} which produces values. * @param timespan @@ -214,12 +215,12 @@ public static Func1>, Subscription> window(Observable * @param scheduler * The {@link rx.Scheduler} to use for timing windows. * @return - * the {@link rx.util.functions.Func1} object representing the specified window operation. + * the {@link rx.util.functions.Func1} object representing the specified window operation. */ - public static Func1>, Subscription> window(final Observable source, final long timespan, final TimeUnit unit, final Scheduler scheduler) { - return new Func1>, Subscription>() { + public static OnSubscribeFunc> window(final Observable source, final long timespan, final TimeUnit unit, final Scheduler scheduler) { + return new OnSubscribeFunc>() { @Override - public Subscription call(final Observer> observer) { + public Subscription onSubscribe(final Observer> observer) { NonOverlappingChunks> windows = new NonOverlappingChunks>(observer, WINDOW_MAKER); ChunkCreator creator = new TimeBasedChunkCreator>(windows, timespan, unit, scheduler); return source.subscribe(new ChunkObserver>(windows, observer, creator)); @@ -234,10 +235,10 @@ public Subscription call(final Observer> observer) { * Additionally the window is automatically emitted once it reaches a specified number of elements. * When the source {@link rx.Observable} completes or produces an error, the current window is emitted, and * the event is propagated to all subscribed {@link rx.Observer}s.

- * + * *

Note that this operation only produces non-overlapping windows. At all times there is * exactly one window actively storing values.

- * + * * @param source * The {@link rx.Observable} which produces values. * @param timespan @@ -247,9 +248,9 @@ public Subscription call(final Observer> observer) { * @param count * The maximum size of the window. Once a window reaches this size, it is emitted. * @return - * the {@link rx.util.functions.Func1} object representing the specified window operation. + * the {@link rx.util.functions.Func1} object representing the specified window operation. */ - public static Func1>, Subscription> window(Observable source, long timespan, TimeUnit unit, int count) { + public static OnSubscribeFunc> window(Observable source, long timespan, TimeUnit unit, int count) { return window(source, timespan, unit, count, Schedulers.threadPoolForComputation()); } @@ -260,10 +261,10 @@ public static Func1>, Subscription> window(Observable * Additionally the window is automatically emitted once it reaches a specified number of elements. * When the source {@link rx.Observable} completes or produces an error, the current window is emitted, and * the event is propagated to all subscribed {@link rx.Observer}s.

- * + * *

Note that this operation only produces non-overlapping windows. At all times there is * exactly one window actively storing values.

- * + * * @param source * The {@link rx.Observable} which produces values. * @param timespan @@ -275,12 +276,12 @@ public static Func1>, Subscription> window(Observable * @param scheduler * The {@link rx.Scheduler} to use for timing windows. * @return - * the {@link rx.util.functions.Func1} object representing the specified window operation. + * the {@link rx.util.functions.Func1} object representing the specified window operation. */ - public static Func1>, Subscription> window(final Observable source, final long timespan, final TimeUnit unit, final int count, final Scheduler scheduler) { - return new Func1>, Subscription>() { + public static OnSubscribeFunc> window(final Observable source, final long timespan, final TimeUnit unit, final int count, final Scheduler scheduler) { + return new OnSubscribeFunc>() { @Override - public Subscription call(final Observer> observer) { + public Subscription onSubscribe(final Observer> observer) { Chunks> chunks = new TimeAndSizeBasedChunks>(observer, WINDOW_MAKER, count, timespan, unit, scheduler); ChunkCreator creator = new SingleChunkCreator>(chunks); return source.subscribe(new ChunkObserver>(chunks, observer, creator)); @@ -295,10 +296,10 @@ public Subscription call(final Observer> observer) { * The creation of windows is also periodical. How often this is done depends on the specified timeshift. * When the source {@link rx.Observable} completes or produces an error, the current window is emitted, and * the event is propagated to all subscribed {@link rx.Observer}s.

- * + * *

Note that this operation can produce non-connected, or overlapping windows depending * on the input parameters.

- * + * * @param source * The {@link rx.Observable} which produces values. * @param timespan @@ -308,9 +309,9 @@ public Subscription call(final Observer> observer) { * @param unit * The {@link java.util.concurrent.TimeUnit} defining the unit of time for the timespan. * @return - * the {@link rx.util.functions.Func1} object representing the specified window operation. + * the {@link rx.util.functions.Func1} object representing the specified window operation. */ - public static Func1>, Subscription> window(Observable source, long timespan, long timeshift, TimeUnit unit) { + public static OnSubscribeFunc> window(Observable source, long timespan, long timeshift, TimeUnit unit) { return window(source, timespan, timeshift, unit, Schedulers.threadPoolForComputation()); } @@ -321,10 +322,10 @@ public static Func1>, Subscription> window(Observable * The creation of windows is also periodical. How often this is done depends on the specified timeshift. * When the source {@link rx.Observable} completes or produces an error, the current window is emitted, and * the event is propagated to all subscribed {@link rx.Observer}s.

- * + * *

Note that this operation can produce non-connected, or overlapping windows depending * on the input parameters.

- * + * * @param source * The {@link rx.Observable} which produces values. * @param timespan @@ -336,12 +337,12 @@ public static Func1>, Subscription> window(Observable * @param scheduler * The {@link rx.Scheduler} to use for timing windows. * @return - * the {@link rx.util.functions.Func1} object representing the specified window operation. + * the {@link rx.util.functions.Func1} object representing the specified window operation. */ - public static Func1>, Subscription> window(final Observable source, final long timespan, final long timeshift, final TimeUnit unit, final Scheduler scheduler) { - return new Func1>, Subscription>() { + public static OnSubscribeFunc> window(final Observable source, final long timespan, final long timeshift, final TimeUnit unit, final Scheduler scheduler) { + return new OnSubscribeFunc>() { @Override - public Subscription call(final Observer> observer) { + public Subscription onSubscribe(final Observer> observer) { OverlappingChunks> windows = new TimeBasedChunks>(observer, WINDOW_MAKER, timespan, unit, scheduler); ChunkCreator creator = new TimeBasedChunkCreator>(windows, timeshift, unit, scheduler); return source.subscribe(new ChunkObserver>(windows, observer, creator)); @@ -351,14 +352,15 @@ public Subscription call(final Observer> observer) { /** * This class represents a single window: A sequence of recorded values. - * - * @param The type of objects which this {@link Window} can hold. + * + * @param + * The type of objects which this {@link Window} can hold. */ protected static class Window extends Chunk> { /** * @return - * The mutable underlying {@link Observable} which contains all the - * recorded values in this {@link Window} object. + * The mutable underlying {@link Observable} which contains all the + * recorded values in this {@link Window} object. */ @Override public Observable getContents() { @@ -453,9 +455,9 @@ public void testTimedAndCount() { final List list = new ArrayList(); final List> lists = new ArrayList>(); - Observable source = Observable.create(new Func1, Subscription>() { + Observable source = Observable.create(new OnSubscribeFunc() { @Override - public Subscription call(Observer observer) { + public Subscription onSubscribe(Observer observer) { push(observer, "one", 10); push(observer, "two", 90); push(observer, "three", 110); @@ -487,9 +489,9 @@ public void testTimed() { final List list = new ArrayList(); final List> lists = new ArrayList>(); - Observable source = Observable.create(new Func1, Subscription>() { + Observable source = Observable.create(new OnSubscribeFunc() { @Override - public Subscription call(Observer observer) { + public Subscription onSubscribe(Observer observer) { push(observer, "one", 98); push(observer, "two", 99); push(observer, "three", 100); @@ -517,9 +519,9 @@ public void testObservableBasedOpenerAndCloser() { final List list = new ArrayList(); final List> lists = new ArrayList>(); - Observable source = Observable.create(new Func1, Subscription>() { + Observable source = Observable.create(new OnSubscribeFunc() { @Override - public Subscription call(Observer observer) { + public Subscription onSubscribe(Observer observer) { push(observer, "one", 10); push(observer, "two", 60); push(observer, "three", 110); @@ -530,9 +532,9 @@ public Subscription call(Observer observer) { } }); - Observable openings = Observable.create(new Func1, Subscription>() { + Observable openings = Observable.create(new OnSubscribeFunc() { @Override - public Subscription call(Observer observer) { + public Subscription onSubscribe(Observer observer) { push(observer, Openings.create(), 50); push(observer, Openings.create(), 200); complete(observer, 250); @@ -543,9 +545,9 @@ public Subscription call(Observer observer) { Func1> closer = new Func1>() { @Override public Observable call(Opening opening) { - return Observable.create(new Func1, Subscription>() { + return Observable.create(new OnSubscribeFunc() { @Override - public Subscription call(Observer observer) { + public Subscription onSubscribe(Observer observer) { push(observer, Closings.create(), 100); complete(observer, 101); return Subscriptions.empty(); @@ -568,9 +570,9 @@ public void testObservableBasedCloser() { final List list = new ArrayList(); final List> lists = new ArrayList>(); - Observable source = Observable.create(new Func1, Subscription>() { + Observable source = Observable.create(new OnSubscribeFunc() { @Override - public Subscription call(Observer observer) { + public Subscription onSubscribe(Observer observer) { push(observer, "one", 10); push(observer, "two", 60); push(observer, "three", 110); @@ -584,9 +586,9 @@ public Subscription call(Observer observer) { Func0> closer = new Func0>() { @Override public Observable call() { - return Observable.create(new Func1, Subscription>() { + return Observable.create(new OnSubscribeFunc() { @Override - public Subscription call(Observer observer) { + public Subscription onSubscribe(Observer observer) { push(observer, Closings.create(), 100); complete(observer, 101); return Subscriptions.empty(); From 959b0d84b4ee762461d47324cd2d4d00a3e49c0f Mon Sep 17 00:00:00 2001 From: jmhofer Date: Thu, 5 Sep 2013 18:51:47 +0200 Subject: [PATCH 3/5] made WINDOW_MAKER typed --- .../java/rx/operators/OperationWindow.java | 27 ++++++++++--------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/rxjava-core/src/main/java/rx/operators/OperationWindow.java b/rxjava-core/src/main/java/rx/operators/OperationWindow.java index 0278fbb4d8..0a24937f3b 100644 --- a/rxjava-core/src/main/java/rx/operators/OperationWindow.java +++ b/rxjava-core/src/main/java/rx/operators/OperationWindow.java @@ -43,12 +43,14 @@ public final class OperationWindow extends ChunkedOperation { - public static final Func0 WINDOW_MAKER = new Func0() { - @Override - public Object call() { - return new Window(); - } - }; + public static Func0> windowMaker() { + return new Func0>() { + @Override + public Window call() { + return new Window(); + } + }; + } /** *

This method creates a {@link rx.util.functions.Func1} object which represents the window operation. This operation takes @@ -74,7 +76,7 @@ public static OnSubscribeFunc> window(final Observable sour return new OnSubscribeFunc>() { @Override public Subscription onSubscribe(final Observer> observer) { - NonOverlappingChunks> windows = new NonOverlappingChunks>(observer, WINDOW_MAKER); + NonOverlappingChunks> windows = new NonOverlappingChunks>(observer, windowMaker()); ChunkCreator creator = new ObservableBasedSingleChunkCreator>(windows, windowClosingSelector); return source.subscribe(new ChunkObserver>(windows, observer, creator)); } @@ -111,7 +113,7 @@ public static OnSubscribeFunc> window(final Observable sour return new OnSubscribeFunc>() { @Override public Subscription onSubscribe(final Observer> observer) { - OverlappingChunks> windows = new OverlappingChunks>(observer, WINDOW_MAKER); + OverlappingChunks> windows = new OverlappingChunks>(observer, windowMaker()); ChunkCreator creator = new ObservableBasedMultiChunkCreator>(windows, windowOpenings, windowClosingSelector); return source.subscribe(new ChunkObserver>(windows, observer, creator)); } @@ -166,7 +168,7 @@ public static OnSubscribeFunc> window(final Observable sour return new OnSubscribeFunc>() { @Override public Subscription onSubscribe(final Observer> observer) { - Chunks> chunks = new SizeBasedChunks>(observer, WINDOW_MAKER, count); + Chunks> chunks = new SizeBasedChunks>(observer, windowMaker(), count); ChunkCreator creator = new SkippingChunkCreator>(chunks, skip); return source.subscribe(new ChunkObserver>(chunks, observer, creator)); } @@ -221,7 +223,7 @@ public static OnSubscribeFunc> window(final Observable sour return new OnSubscribeFunc>() { @Override public Subscription onSubscribe(final Observer> observer) { - NonOverlappingChunks> windows = new NonOverlappingChunks>(observer, WINDOW_MAKER); + NonOverlappingChunks> windows = new NonOverlappingChunks>(observer, windowMaker()); ChunkCreator creator = new TimeBasedChunkCreator>(windows, timespan, unit, scheduler); return source.subscribe(new ChunkObserver>(windows, observer, creator)); } @@ -282,7 +284,7 @@ public static OnSubscribeFunc> window(final Observable sour return new OnSubscribeFunc>() { @Override public Subscription onSubscribe(final Observer> observer) { - Chunks> chunks = new TimeAndSizeBasedChunks>(observer, WINDOW_MAKER, count, timespan, unit, scheduler); + Chunks> chunks = new TimeAndSizeBasedChunks>(observer, windowMaker(), count, timespan, unit, scheduler); ChunkCreator creator = new SingleChunkCreator>(chunks); return source.subscribe(new ChunkObserver>(chunks, observer, creator)); } @@ -343,7 +345,7 @@ public static OnSubscribeFunc> window(final Observable sour return new OnSubscribeFunc>() { @Override public Subscription onSubscribe(final Observer> observer) { - OverlappingChunks> windows = new TimeBasedChunks>(observer, WINDOW_MAKER, timespan, unit, scheduler); + OverlappingChunks> windows = new TimeBasedChunks>(observer, windowMaker(), timespan, unit, scheduler); ChunkCreator creator = new TimeBasedChunkCreator>(windows, timeshift, unit, scheduler); return source.subscribe(new ChunkObserver>(windows, observer, creator)); } @@ -373,7 +375,6 @@ public static class UnitTest { private TestScheduler scheduler; @Before - @SuppressWarnings("unchecked") public void before() { scheduler = new TestScheduler(); } From a7b40b3b5a0ec673694291f4aab9fa89a5750014 Mon Sep 17 00:00:00 2001 From: jmhofer Date: Thu, 5 Sep 2013 19:25:35 +0200 Subject: [PATCH 4/5] removed raw Func0s from the chunks --- .../main/java/rx/operators/ChunkedOperation.java | 14 +++++++------- .../main/java/rx/operators/OperationWindow.java | 12 ++++++------ 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/rxjava-core/src/main/java/rx/operators/ChunkedOperation.java b/rxjava-core/src/main/java/rx/operators/ChunkedOperation.java index 5b5eb3344c..f0990425bb 100644 --- a/rxjava-core/src/main/java/rx/operators/ChunkedOperation.java +++ b/rxjava-core/src/main/java/rx/operators/ChunkedOperation.java @@ -105,7 +105,7 @@ protected static class NonOverlappingChunks extends Chunks { private final Object lock = new Object(); - public NonOverlappingChunks(Observer observer, Func0 chunkMaker) { + public NonOverlappingChunks(Observer observer, Func0> chunkMaker) { super(observer, chunkMaker); } @@ -134,7 +134,7 @@ public void pushValue(T value) { * The type of object being tracked by the {@link rx.operators.ChunkedOperation.Chunk} */ protected static class OverlappingChunks extends Chunks { - public OverlappingChunks(Observer observer, Func0 chunkMaker) { + public OverlappingChunks(Observer observer, Func0> chunkMaker) { super(observer, chunkMaker); } } @@ -158,7 +158,7 @@ protected static class TimeAndSizeBasedChunks extends Chunks { private final TimeUnit unit; private final int maxSize; - public TimeAndSizeBasedChunks(Observer observer, Func0 chunkMaker, int maxSize, long maxTime, TimeUnit unit, Scheduler scheduler) { + public TimeAndSizeBasedChunks(Observer observer, Func0> chunkMaker, int maxSize, long maxTime, TimeUnit unit, Scheduler scheduler) { super(observer, chunkMaker); this.maxSize = maxSize; this.maxTime = maxTime; @@ -224,7 +224,7 @@ protected static class TimeBasedChunks extends OverlappingChunks { private final long time; private final TimeUnit unit; - public TimeBasedChunks(Observer observer, Func0 chunkMaker, long time, TimeUnit unit, Scheduler scheduler) { + public TimeBasedChunks(Observer observer, Func0> chunkMaker, long time, TimeUnit unit, Scheduler scheduler) { super(observer, chunkMaker); this.time = time; this.unit = unit; @@ -263,7 +263,7 @@ protected static class SizeBasedChunks extends Chunks { private final int size; - public SizeBasedChunks(Observer observer, Func0 chunkMaker, int size) { + public SizeBasedChunks(Observer observer, Func0> chunkMaker, int size) { super(observer, chunkMaker); this.size = size; } @@ -295,7 +295,7 @@ protected static class Chunks { private final Queue> chunks = new ConcurrentLinkedQueue>(); private final Observer observer; - private final Func0> chunkMaker; + private final Func0> chunkMaker; /** * Constructs a new {@link ChunkedOperation.Chunks} object for the specified {@link rx.Observer}. @@ -303,7 +303,7 @@ protected static class Chunks { * @param observer * The {@link rx.Observer} to which this object will emit its internal {@link rx.operators.ChunkedOperation.Chunk} objects to when requested. */ - public Chunks(Observer observer, Func0 chunkMaker) { + public Chunks(Observer observer, Func0> chunkMaker) { this.observer = observer; this.chunkMaker = chunkMaker; } diff --git a/rxjava-core/src/main/java/rx/operators/OperationWindow.java b/rxjava-core/src/main/java/rx/operators/OperationWindow.java index 0a24937f3b..ab15ed2352 100644 --- a/rxjava-core/src/main/java/rx/operators/OperationWindow.java +++ b/rxjava-core/src/main/java/rx/operators/OperationWindow.java @@ -76,7 +76,7 @@ public static OnSubscribeFunc> window(final Observable sour return new OnSubscribeFunc>() { @Override public Subscription onSubscribe(final Observer> observer) { - NonOverlappingChunks> windows = new NonOverlappingChunks>(observer, windowMaker()); + NonOverlappingChunks> windows = new NonOverlappingChunks>(observer, OperationWindow.windowMaker()); ChunkCreator creator = new ObservableBasedSingleChunkCreator>(windows, windowClosingSelector); return source.subscribe(new ChunkObserver>(windows, observer, creator)); } @@ -113,7 +113,7 @@ public static OnSubscribeFunc> window(final Observable sour return new OnSubscribeFunc>() { @Override public Subscription onSubscribe(final Observer> observer) { - OverlappingChunks> windows = new OverlappingChunks>(observer, windowMaker()); + OverlappingChunks> windows = new OverlappingChunks>(observer, OperationWindow.windowMaker()); ChunkCreator creator = new ObservableBasedMultiChunkCreator>(windows, windowOpenings, windowClosingSelector); return source.subscribe(new ChunkObserver>(windows, observer, creator)); } @@ -168,7 +168,7 @@ public static OnSubscribeFunc> window(final Observable sour return new OnSubscribeFunc>() { @Override public Subscription onSubscribe(final Observer> observer) { - Chunks> chunks = new SizeBasedChunks>(observer, windowMaker(), count); + Chunks> chunks = new SizeBasedChunks>(observer, OperationWindow.windowMaker(), count); ChunkCreator creator = new SkippingChunkCreator>(chunks, skip); return source.subscribe(new ChunkObserver>(chunks, observer, creator)); } @@ -223,7 +223,7 @@ public static OnSubscribeFunc> window(final Observable sour return new OnSubscribeFunc>() { @Override public Subscription onSubscribe(final Observer> observer) { - NonOverlappingChunks> windows = new NonOverlappingChunks>(observer, windowMaker()); + NonOverlappingChunks> windows = new NonOverlappingChunks>(observer, OperationWindow.windowMaker()); ChunkCreator creator = new TimeBasedChunkCreator>(windows, timespan, unit, scheduler); return source.subscribe(new ChunkObserver>(windows, observer, creator)); } @@ -284,7 +284,7 @@ public static OnSubscribeFunc> window(final Observable sour return new OnSubscribeFunc>() { @Override public Subscription onSubscribe(final Observer> observer) { - Chunks> chunks = new TimeAndSizeBasedChunks>(observer, windowMaker(), count, timespan, unit, scheduler); + Chunks> chunks = new TimeAndSizeBasedChunks>(observer, OperationWindow.windowMaker(), count, timespan, unit, scheduler); ChunkCreator creator = new SingleChunkCreator>(chunks); return source.subscribe(new ChunkObserver>(chunks, observer, creator)); } @@ -345,7 +345,7 @@ public static OnSubscribeFunc> window(final Observable sour return new OnSubscribeFunc>() { @Override public Subscription onSubscribe(final Observer> observer) { - OverlappingChunks> windows = new TimeBasedChunks>(observer, windowMaker(), timespan, unit, scheduler); + OverlappingChunks> windows = new TimeBasedChunks>(observer, OperationWindow.windowMaker(), timespan, unit, scheduler); ChunkCreator creator = new TimeBasedChunkCreator>(windows, timeshift, unit, scheduler); return source.subscribe(new ChunkObserver>(windows, observer, creator)); } From 92fe7f9d32f2c53767ecfea631aae79c9f3ac9d4 Mon Sep 17 00:00:00 2001 From: jmhofer Date: Thu, 5 Sep 2013 19:47:07 +0200 Subject: [PATCH 5/5] made the public window methods more generic via the basic (lol) super/extends fluff; also simplified api by removing a few useless super definitions (there's no super of Opening and Closing) --- rxjava-core/src/main/java/rx/Observable.java | 22 +++++++-------- .../java/rx/operators/ChunkedOperation.java | 2 +- .../java/rx/operators/OperationBuffer.java | 28 ++++++++++--------- .../java/rx/operators/OperationWindow.java | 20 ++++++------- 4 files changed, 37 insertions(+), 35 deletions(-) diff --git a/rxjava-core/src/main/java/rx/Observable.java b/rxjava-core/src/main/java/rx/Observable.java index 25073ca729..e4b64fc28a 100644 --- a/rxjava-core/src/main/java/rx/Observable.java +++ b/rxjava-core/src/main/java/rx/Observable.java @@ -1337,7 +1337,7 @@ public Observable> buffer(Func0> * @return * An {@link Observable} which produces buffers which are created and emitted when the specified {@link Observable}s publish certain objects. */ - public Observable> buffer(Observable bufferOpenings, Func1> bufferClosingSelector) { + public Observable> buffer(Observable bufferOpenings, Func1> bufferClosingSelector) { return create(OperationBuffer.buffer(this, bufferOpenings, bufferClosingSelector)); } @@ -1520,7 +1520,7 @@ public Observable> buffer(long timespan, long timeshift, TimeUnit unit, * An {@link Observable} which produces connected non-overlapping windows, which are emitted * when the current {@link Observable} created with the {@link Func0} argument produces a {@link rx.util.Closing} object. */ - public Observable> window(Observable source, Func0> closingSelector) { + public Observable> window(Observable source, Func0> closingSelector) { return create(OperationWindow.window(source, closingSelector)); } @@ -1542,7 +1542,7 @@ public Observable> window(Observable source, Func0> window(Observable source, Observable windowOpenings, Func1> closingSelector) { + public Observable> window(Observable source, Observable windowOpenings, Func1> closingSelector) { return create(OperationWindow.window(source, windowOpenings, closingSelector)); } @@ -1559,7 +1559,7 @@ public Observable> window(Observable source, Observable> window(Observable source, int count) { + public Observable> window(Observable source, int count) { return create(OperationWindow.window(source, count)); } @@ -1579,7 +1579,7 @@ public Observable> window(Observable source, int count) { * An {@link Observable} which produces windows every "skipped" values containing at most * "count" produced values. */ - public Observable> window(Observable source, int count, int skip) { + public Observable> window(Observable source, int count, int skip) { return create(OperationWindow.window(source, count, skip)); } @@ -1598,7 +1598,7 @@ public Observable> window(Observable source, int count, int ski * @return * An {@link Observable} which produces connected non-overlapping windows with a fixed duration. */ - public Observable> window(Observable source, long timespan, TimeUnit unit) { + public Observable> window(Observable source, long timespan, TimeUnit unit) { return create(OperationWindow.window(source, timespan, unit)); } @@ -1619,7 +1619,7 @@ public Observable> window(Observable source, long timespan, Tim * @return * An {@link Observable} which produces connected non-overlapping windows with a fixed duration. */ - public Observable> window(Observable source, long timespan, TimeUnit unit, Scheduler scheduler) { + public Observable> window(Observable source, long timespan, TimeUnit unit, Scheduler scheduler) { return create(OperationWindow.window(source, timespan, unit, scheduler)); } @@ -1642,7 +1642,7 @@ public Observable> window(Observable source, long timespan, Tim * An {@link Observable} which produces connected non-overlapping windows which are emitted after * a fixed duration or when the window has reached maximum capacity (which ever occurs first). */ - public Observable> window(Observable source, long timespan, TimeUnit unit, int count) { + public Observable> window(Observable source, long timespan, TimeUnit unit, int count) { return create(OperationWindow.window(source, timespan, unit, count)); } @@ -1667,7 +1667,7 @@ public Observable> window(Observable source, long timespan, Tim * An {@link Observable} which produces connected non-overlapping windows which are emitted after * a fixed duration or when the window has reached maximum capacity (which ever occurs first). */ - public Observable> window(Observable source, long timespan, TimeUnit unit, int count, Scheduler scheduler) { + public Observable> window(Observable source, long timespan, TimeUnit unit, int count, Scheduler scheduler) { return create(OperationWindow.window(source, timespan, unit, count, scheduler)); } @@ -1689,7 +1689,7 @@ public Observable> window(Observable source, long timespan, Tim * An {@link Observable} which produces new windows periodically, and these are emitted after * a fixed timespan has elapsed. */ - public Observable> window(Observable source, long timespan, long timeshift, TimeUnit unit) { + public Observable> window(Observable source, long timespan, long timeshift, TimeUnit unit) { return create(OperationWindow.window(source, timespan, timeshift, unit)); } @@ -1713,7 +1713,7 @@ public Observable> window(Observable source, long timespan, lon * An {@link Observable} which produces new windows periodically, and these are emitted after * a fixed timespan has elapsed. */ - public Observable> window(Observable source, long timespan, long timeshift, TimeUnit unit, Scheduler scheduler) { + public Observable> window(Observable source, long timespan, long timeshift, TimeUnit unit, Scheduler scheduler) { return create(OperationWindow.window(source, timespan, timeshift, unit, scheduler)); } diff --git a/rxjava-core/src/main/java/rx/operators/ChunkedOperation.java b/rxjava-core/src/main/java/rx/operators/ChunkedOperation.java index f0990425bb..1b5cf8d0d1 100644 --- a/rxjava-core/src/main/java/rx/operators/ChunkedOperation.java +++ b/rxjava-core/src/main/java/rx/operators/ChunkedOperation.java @@ -495,7 +495,7 @@ protected static class ObservableBasedMultiChunkCreator implements ChunkCr private final SafeObservableSubscription subscription = new SafeObservableSubscription(); - public ObservableBasedMultiChunkCreator(final OverlappingChunks chunks, Observable openings, final Func1> chunkClosingSelector) { + public ObservableBasedMultiChunkCreator(final OverlappingChunks chunks, Observable openings, final Func1> chunkClosingSelector) { subscription.wrap(openings.subscribe(new Action1() { @Override public void call(Opening opening) { diff --git a/rxjava-core/src/main/java/rx/operators/OperationBuffer.java b/rxjava-core/src/main/java/rx/operators/OperationBuffer.java index c692ed6cee..1cc559fb57 100644 --- a/rxjava-core/src/main/java/rx/operators/OperationBuffer.java +++ b/rxjava-core/src/main/java/rx/operators/OperationBuffer.java @@ -42,12 +42,14 @@ public final class OperationBuffer extends ChunkedOperation { - private static final Func0 BUFFER_MAKER = new Func0() { - @Override - public Object call() { - return new Buffer(); - } - }; + private static Func0> bufferMaker() { + return new Func0>() { + @Override + public Buffer call() { + return new Buffer(); + } + }; + } /** *

This method creates a {@link Func1} object which represents the buffer operation. This operation takes @@ -74,7 +76,7 @@ public static OnSubscribeFunc> buffer(final Observable source, fi @Override public Subscription onSubscribe(Observer> observer) { - NonOverlappingChunks> buffers = new NonOverlappingChunks>(observer, BUFFER_MAKER); + NonOverlappingChunks> buffers = new NonOverlappingChunks>(observer, OperationBuffer.bufferMaker()); ChunkCreator creator = new ObservableBasedSingleChunkCreator>(buffers, bufferClosingSelector); return source.subscribe(new ChunkObserver>(buffers, observer, creator)); } @@ -106,11 +108,11 @@ public Subscription onSubscribe(Observer> observer) { * @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, BUFFER_MAKER); + OverlappingChunks> buffers = new OverlappingChunks>(observer, OperationBuffer.bufferMaker()); ChunkCreator creator = new ObservableBasedMultiChunkCreator>(buffers, bufferOpenings, bufferClosingSelector); return source.subscribe(new ChunkObserver>(buffers, observer, creator)); } @@ -165,7 +167,7 @@ public static OnSubscribeFunc> buffer(final Observable source, fi return new OnSubscribeFunc>() { @Override public Subscription onSubscribe(final Observer> observer) { - Chunks> chunks = new SizeBasedChunks>(observer, BUFFER_MAKER, count); + Chunks> chunks = new SizeBasedChunks>(observer, OperationBuffer.bufferMaker(), count); ChunkCreator creator = new SkippingChunkCreator>(chunks, skip); return source.subscribe(new ChunkObserver>(chunks, observer, creator)); } @@ -220,7 +222,7 @@ public static OnSubscribeFunc> buffer(final Observable source, fi return new OnSubscribeFunc>() { @Override public Subscription onSubscribe(final Observer> observer) { - NonOverlappingChunks> buffers = new NonOverlappingChunks>(observer, BUFFER_MAKER); + NonOverlappingChunks> buffers = new NonOverlappingChunks>(observer, OperationBuffer.bufferMaker()); ChunkCreator creator = new TimeBasedChunkCreator>(buffers, timespan, unit, scheduler); return source.subscribe(new ChunkObserver>(buffers, observer, creator)); } @@ -281,7 +283,7 @@ public static OnSubscribeFunc> buffer(final Observable source, fi return new OnSubscribeFunc>() { @Override public Subscription onSubscribe(final Observer> observer) { - Chunks> chunks = new TimeAndSizeBasedChunks>(observer, BUFFER_MAKER, count, timespan, unit, scheduler); + Chunks> chunks = new TimeAndSizeBasedChunks>(observer, OperationBuffer.bufferMaker(), count, timespan, unit, scheduler); ChunkCreator creator = new SingleChunkCreator>(chunks); return source.subscribe(new ChunkObserver>(chunks, observer, creator)); } @@ -342,7 +344,7 @@ public static OnSubscribeFunc> buffer(final Observable source, fi return new OnSubscribeFunc>() { @Override public Subscription onSubscribe(final Observer> observer) { - OverlappingChunks> buffers = new TimeBasedChunks>(observer, BUFFER_MAKER, timespan, unit, scheduler); + OverlappingChunks> buffers = new TimeBasedChunks>(observer, OperationBuffer.bufferMaker(), timespan, unit, scheduler); ChunkCreator creator = new TimeBasedChunkCreator>(buffers, timeshift, unit, scheduler); 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 ab15ed2352..5cffda9661 100644 --- a/rxjava-core/src/main/java/rx/operators/OperationWindow.java +++ b/rxjava-core/src/main/java/rx/operators/OperationWindow.java @@ -72,7 +72,7 @@ public Window call() { * @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) { @@ -109,7 +109,7 @@ public Subscription onSubscribe(final Observer> observer) * @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) { @@ -137,7 +137,7 @@ public Subscription onSubscribe(final Observer> observer) * @return * the {@link rx.util.functions.Func1} object representing the specified window operation. */ - public static OnSubscribeFunc> window(Observable source, int count) { + public static OnSubscribeFunc> window(Observable source, int count) { return window(source, count, count); } @@ -164,7 +164,7 @@ public static OnSubscribeFunc> window(Observable source, in * @return * the {@link rx.util.functions.Func1} object representing the specified window operation. */ - public static OnSubscribeFunc> window(final Observable source, final int count, final int skip) { + public static OnSubscribeFunc> window(final Observable source, final int count, final int skip) { return new OnSubscribeFunc>() { @Override public Subscription onSubscribe(final Observer> observer) { @@ -194,7 +194,7 @@ public Subscription onSubscribe(final Observer> observer) * @return * the {@link rx.util.functions.Func1} object representing the specified window operation. */ - public static OnSubscribeFunc> window(Observable source, long timespan, TimeUnit unit) { + public static OnSubscribeFunc> window(Observable source, long timespan, TimeUnit unit) { return window(source, timespan, unit, Schedulers.threadPoolForComputation()); } @@ -219,7 +219,7 @@ public static OnSubscribeFunc> window(Observable source, lo * @return * the {@link rx.util.functions.Func1} object representing the specified window operation. */ - public static OnSubscribeFunc> window(final Observable source, final long timespan, final TimeUnit unit, final Scheduler scheduler) { + public static OnSubscribeFunc> window(final Observable source, final long timespan, final TimeUnit unit, final Scheduler scheduler) { return new OnSubscribeFunc>() { @Override public Subscription onSubscribe(final Observer> observer) { @@ -252,7 +252,7 @@ public Subscription onSubscribe(final Observer> observer) * @return * the {@link rx.util.functions.Func1} object representing the specified window operation. */ - public static OnSubscribeFunc> window(Observable source, long timespan, TimeUnit unit, int count) { + public static OnSubscribeFunc> window(Observable source, long timespan, TimeUnit unit, int count) { return window(source, timespan, unit, count, Schedulers.threadPoolForComputation()); } @@ -280,7 +280,7 @@ public static OnSubscribeFunc> window(Observable source, lo * @return * the {@link rx.util.functions.Func1} object representing the specified window operation. */ - public static OnSubscribeFunc> window(final Observable source, final long timespan, final TimeUnit unit, final int count, final Scheduler scheduler) { + public static OnSubscribeFunc> window(final Observable source, final long timespan, final TimeUnit unit, final int count, final Scheduler scheduler) { return new OnSubscribeFunc>() { @Override public Subscription onSubscribe(final Observer> observer) { @@ -313,7 +313,7 @@ public Subscription onSubscribe(final Observer> observer) * @return * the {@link rx.util.functions.Func1} object representing the specified window operation. */ - public static OnSubscribeFunc> window(Observable source, long timespan, long timeshift, TimeUnit unit) { + public static OnSubscribeFunc> window(Observable source, long timespan, long timeshift, TimeUnit unit) { return window(source, timespan, timeshift, unit, Schedulers.threadPoolForComputation()); } @@ -341,7 +341,7 @@ public static OnSubscribeFunc> window(Observable source, lo * @return * the {@link rx.util.functions.Func1} object representing the specified window operation. */ - public static OnSubscribeFunc> window(final Observable source, final long timespan, final long timeshift, final TimeUnit unit, final Scheduler scheduler) { + public static OnSubscribeFunc> window(final Observable source, final long timespan, final long timeshift, final TimeUnit unit, final Scheduler scheduler) { return new OnSubscribeFunc>() { @Override public Subscription onSubscribe(final Observer> observer) {