From 009b3e2a0b86b652b53bce5301150cb925e2dd65 Mon Sep 17 00:00:00 2001 From: Nitesh Kant Date: Wed, 1 Jul 2015 18:25:08 -0700 Subject: [PATCH 01/81] New operators: `concatEmptyWith` and `mergeEmptyWith`. As discussed in issue #3037, the primary use of these operators is to be applied to `Observable` so that they can be merged and concatenated with an Observable of a different type. Both these operators raise an error if the source Observable emits any item. Review comments --- src/main/java/rx/Observable.java | 69 ++++++ .../operators/OperatorConcatEmptyWith.java | 198 ++++++++++++++++++ .../operators/OperatorMergeEmptyWith.java | 122 +++++++++++ .../OperatorConcatEmptyWithTest.java | 167 +++++++++++++++ .../operators/OperatorMergeEmptyWithTest.java | 145 +++++++++++++ 5 files changed, 701 insertions(+) create mode 100644 src/main/java/rx/internal/operators/OperatorConcatEmptyWith.java create mode 100644 src/main/java/rx/internal/operators/OperatorMergeEmptyWith.java create mode 100644 src/test/java/rx/internal/operators/OperatorConcatEmptyWithTest.java create mode 100644 src/test/java/rx/internal/operators/OperatorMergeEmptyWithTest.java diff --git a/src/main/java/rx/Observable.java b/src/main/java/rx/Observable.java index 3f27cb0a83..719f1dcf5c 100644 --- a/src/main/java/rx/Observable.java +++ b/src/main/java/rx/Observable.java @@ -9467,6 +9467,75 @@ public final Observable zipWith(Observable other, Func2 return zip(this, other, zipFunction); } + /** + * Returns an Observable that upon completion of the source Observable subscribes to the passed {@code other} + * Observable and then emits all items emitted by that Observable. This function does not expect the source + * Observable to emit any item, in case, the source Observable, emits any item, an {@link IllegalStateException} + * is raised. + *

+ * + * This is different than {@link #concatWith(Observable)} as it does not expect the source Observable to ever emit + * an item. So, this usually is useful for {@code Observable} and results in cleaner code as opposed to using + * a {@link #cast(Class)}, something like: + * + * {@code Observable.empty().cast(String.class).concatWith(Observable.just("Hello"))} + * + *

+ *
Scheduler:
+ *
{@code concatEmptyWith} does not operate by default on a particular {@link Scheduler}.
+ *
+ * + *
Backpressure:
+ *
{@code concatEmptyWith} does not propagate any demands from the subscriber to the source {@code Observable} + * as it never expects the source to ever emit an item. All demands are sent to the {@code other} + * {@code Observable}.
+ * + * @return an Observable that upon completion of the source, starts emitting items from the {@code other} + * Observable. + * @throws IllegalStateException If the source emits any item. + * + * @see #mergeEmptyWith(Observable) + */ + @Experimental + public final Observable concatEmptyWith(Observable other) { + return lift(new OperatorConcatEmptyWith(other)); + } + + /** + * Returns an Observable that only listens for error from the source Observable and emit items only from the passed + * {@code other} Observable. This function does not expect the source Observable to emit any item, in case, the + * source Observable, emits any item, an {@link IllegalStateException} is raised. + *

+ * + * This is different than {@link #mergeWith(Observable)} as it does not expect the source Observable to ever emit + * an item. So, this usually is useful for using on {@code Observable} and results in cleaner code as opposed + * to using a {@link #cast(Class)}, something like: + * {@code Observable.empty().cast(String.class).mergeWith(Observable.just("Hello"))} + * + *

+ *
Scheduler:
+ *
{@code mergeEmptyWith} does not operate by default on a particular {@link Scheduler}.
+ *
+ * + *
+ *
Backpressure:
+ *
{@code mergeEmptyWith} does not propagate any demands from the subscriber to the source {@code Observable} + * as it never expects the source to ever emit an item. All demands are sent to the {@code other} + * {@code Observable}.
+ *
+ * + * @return an Observable that only listens for errors from the source and starts emitting items from the + * {@code other} Observable on subscription. + * Observable. + * @throws IllegalStateException If the source emits any item. + * + * @see #concatEmptyWith(Observable) + */ + @Experimental + public final Observable mergeEmptyWith(Observable other) { + return lift(new OperatorMergeEmptyWith(other)); + } + /** * An Observable that never sends any information to an {@link Observer}. * This Observable is useful primarily for testing purposes. diff --git a/src/main/java/rx/internal/operators/OperatorConcatEmptyWith.java b/src/main/java/rx/internal/operators/OperatorConcatEmptyWith.java new file mode 100644 index 0000000000..0a3649eb56 --- /dev/null +++ b/src/main/java/rx/internal/operators/OperatorConcatEmptyWith.java @@ -0,0 +1,198 @@ +/** + * Copyright 2014 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.internal.operators; + +import rx.Observable; +import rx.Observable.Operator; +import rx.Producer; +import rx.Subscriber; +import rx.internal.producers.ProducerArbiter; +import rx.subscriptions.SerialSubscription; + +/** + * Returns an Observable that emits an error if any item is emitted by the source and emits items from the supplied + * alternate {@code Observable} after the source completes. + * + * @param the source value type + * @param the result value type + */ +public final class OperatorConcatEmptyWith implements Operator { + + private final Observable alternate; + + public OperatorConcatEmptyWith(Observable alternate) { + this.alternate = alternate; + } + + @Override + public Subscriber call(Subscriber child) { + final SerialSubscription ssub = new SerialSubscription(); + final ParentSubscriber parent = new ParentSubscriber(child, ssub, alternate); + ssub.set(parent); + child.add(ssub); + child.setProducer(parent.emptyProducer); + return parent; + } + + private final class ParentSubscriber extends Subscriber { + + private final Subscriber child; + private final SerialSubscription ssub; + private final EmptyProducer emptyProducer; + private final Observable alternate; + + ParentSubscriber(Subscriber child, final SerialSubscription ssub, Observable alternate) { + this.child = child; + this.ssub = ssub; + this.emptyProducer = new EmptyProducer(); + this.alternate = alternate; + } + + @Override + public void setProducer(final Producer producer) { + /* + * Always request Max from the parent as we never really expect the parent to emit an item, so the + * actual value does not matter. However, if the parent producer is waiting for a request to emit + * a terminal event, not requesting the same will cause a deadlock of the parent never completing and + * the child never subscribed. + */ + producer.request(Long.MAX_VALUE); + } + + @Override + public void onCompleted() { + if (!child.isUnsubscribed()) { + AlternateSubscriber as = new AlternateSubscriber(child, emptyProducer); + ssub.set(as); + alternate.unsafeSubscribe(as); + } + } + + @Override + public void onError(Throwable e) { + child.onError(e); + } + + @Override + public void onNext(T t) { + onError(new IllegalStateException("Concat empty with source emitted an item: " + t)); + } + } + + private final class AlternateSubscriber extends Subscriber { + + private final EmptyProducer emptyProducer; + private final Subscriber child; + + AlternateSubscriber(Subscriber child, EmptyProducer emptyProducer) { + this.child = child; + this.emptyProducer = emptyProducer; + } + + @Override + public void setProducer(final Producer producer) { + emptyProducer.setAltProducer(producer); + } + + @Override + public void onCompleted() { + child.onCompleted(); + } + + @Override + public void onError(Throwable e) { + child.onError(e); + } + + @Override + public void onNext(R r) { + child.onNext(r); + } + } + + /** + * This is a producer implementation that does the following: + * + *
    + *
  • If the alternate producer has not yet arrived, store the total requested count from downstream.
  • + *
  • If the alternate producer has arrived, then relay the request demand to it.
  • + *
  • Request {@link Long#MAX_VALUE} from the parent producer, the first time the child requests anything.
  • + *
+ * + * Since, this is only applicable to this operator, it does not check for emissions from the source, as the source + * is never expected to emit any item. Thus it is "lighter" weight than {@link ProducerArbiter} + */ + private static final class EmptyProducer implements Producer { + + /*Total requested items till the time the alternate producer arrives.*/ + private long missedRequested; /*Guarded by this*/ + /*Producer from the alternate Observable for this operator*/ + private Producer altProducer; /*Guarded by this*/ + + @Override + public void request(final long requested) { + if (requested < 0) { + throw new IllegalArgumentException("Requested items can not be negative."); + } + + if (requested == 0) { + return; + } + + boolean requestToAlternate = false; + + synchronized (this) { + if (null == altProducer) { + /*Accumulate requested till the time an alternate producer arrives.*/ + long r = this.missedRequested; + long u = r + requested; + if (u < 0) { + u = Long.MAX_VALUE; + } + this.missedRequested = u; + } else { + /*If the alternate producer exists, then relay a valid request. The missed requested will be + requested from the alt producer on setProducer()*/ + requestToAlternate = true; + } + } + + if (requestToAlternate) { + altProducer.request(requested); + } + } + + private void setAltProducer(Producer altProducer) { + if (null == altProducer) { + throw new IllegalArgumentException("Producer can not be null."); + } + + boolean requestToAlternate = false; + + synchronized (this) { + if (0 != missedRequested) { + /*Something was requested from the source Observable, relay that to the new producer*/ + requestToAlternate = true; + } + this.altProducer = altProducer; + } + + if (requestToAlternate) { + this.altProducer.request(missedRequested); + } + } + } +} diff --git a/src/main/java/rx/internal/operators/OperatorMergeEmptyWith.java b/src/main/java/rx/internal/operators/OperatorMergeEmptyWith.java new file mode 100644 index 0000000000..3d80cd1a88 --- /dev/null +++ b/src/main/java/rx/internal/operators/OperatorMergeEmptyWith.java @@ -0,0 +1,122 @@ +/** + * Copyright 2014 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.internal.operators; + +import rx.Observable; +import rx.Observable.Operator; +import rx.Producer; +import rx.Subscriber; +import rx.observers.SerializedSubscriber; + +/** + * Returns an Observable that emits an error if any item is emitted by the source and emits items from the supplied + * alternate {@code Observable}. The errors from source are propagated as-is. + * + * @param the source value type + * @param the result value type + */ +public final class OperatorMergeEmptyWith implements Operator { + + private final Observable alternate; + + public OperatorMergeEmptyWith(Observable alternate) { + this.alternate = alternate; + } + + @Override + public Subscriber call(final Subscriber child) { + final ChildSubscriber wrappedChild = new ChildSubscriber(child); + final ParentSubscriber parent = new ParentSubscriber(wrappedChild); + wrappedChild.add(parent); + alternate.unsafeSubscribe(wrappedChild); + return parent; + } + + private final class ParentSubscriber extends Subscriber { + + private final ChildSubscriber child; + + ParentSubscriber(ChildSubscriber child) { + this.child = child; + } + + @Override + public void setProducer(final Producer producer) { + /* + * Always request Max from the parent as we never really expect the parent to emit an item, so the + * actual value does not matter. However, if the parent producer is waiting for a request to emit + * a terminal event, not requesting the same will cause the merged Observable to never complete. + */ + producer.request(Long.MAX_VALUE); + } + + @Override + public void onCompleted() { + child.parentCompleted(); + } + + @Override + public void onError(Throwable e) { + child.onError(e); + } + + @Override + public void onNext(T t) { + onError(new IllegalStateException("Merge empty with source emitted an item: " + t)); + } + } + + private final class ChildSubscriber extends SerializedSubscriber { + + private final Subscriber delegate; + private boolean parentCompleted; /*Guarded by this*/ + private boolean childCompleted; /*Guarded by this*/ + + ChildSubscriber(Subscriber delegate) { + super(delegate); + this.delegate = delegate; + } + + @Override + public void onCompleted() { + synchronized (this) { + if (parentCompleted) { + delegate.onCompleted(); + } + childCompleted = true; + } + } + + @Override + public void onError(Throwable e) { + delegate.onError(e); + } + + @Override + public void onNext(R r) { + delegate.onNext(r); + } + + public void parentCompleted() { + synchronized (this) { + if (childCompleted) { + delegate.onCompleted(); + } + parentCompleted = true; + } + } + } +} diff --git a/src/test/java/rx/internal/operators/OperatorConcatEmptyWithTest.java b/src/test/java/rx/internal/operators/OperatorConcatEmptyWithTest.java new file mode 100644 index 0000000000..bf7bf35e32 --- /dev/null +++ b/src/test/java/rx/internal/operators/OperatorConcatEmptyWithTest.java @@ -0,0 +1,167 @@ +/** + * Copyright 2014 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.internal.operators; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import rx.Observable; +import rx.Observable.OnSubscribe; +import rx.Producer; +import rx.Subscriber; +import rx.observers.TestSubscriber; +import rx.schedulers.Schedulers; +import rx.schedulers.TestScheduler; + +public class OperatorConcatEmptyWithTest { + + @Rule + public final ExpectedException expectedException = ExpectedException.none(); + + @Test(timeout = 60000) + public void testWithVoid() { + final String soleValue = "Hello"; + Observable source = Observable.empty() + .concatEmptyWith(Observable.just(soleValue)); + + TestSubscriber testSubscriber = new TestSubscriber(); + source.subscribe(testSubscriber); + + testSubscriber.awaitTerminalEvent(); + testSubscriber.assertNoErrors(); + testSubscriber.assertValue(soleValue); + } + + @Test(timeout = 60000) + public void testErrorOnSourceEmitItem() { + Observable source = Observable.just(1) + .concatEmptyWith(Observable.just("Hello")); + + TestSubscriber testSubscriber = new TestSubscriber(); + source.subscribe(testSubscriber); + + testSubscriber.awaitTerminalEvent(); + testSubscriber.assertNoValues(); + testSubscriber.assertError(IllegalStateException.class); + } + + @Test(timeout = 60000) + public void testSourceError() throws Exception { + Observable source = Observable.error(new IllegalStateException()) + .concatEmptyWith(Observable.just("Hello")); + + TestSubscriber testSubscriber = new TestSubscriber(); + source.subscribe(testSubscriber); + + testSubscriber.awaitTerminalEvent(); + testSubscriber.assertNoValues(); + testSubscriber.assertError(IllegalStateException.class); + } + + @Test(timeout = 60000) + public void testNoSubscribeBeforeSourceCompletion() { + final String soleValue = "Hello"; + final TestScheduler testScheduler = Schedulers.test(); + + /*Delaying on complete event so to check that the subscription does not happen before completion*/ + Observable source = Observable.empty() + .observeOn(testScheduler) + .concatEmptyWith(Observable.just(soleValue)); + + TestSubscriber testSubscriber = new TestSubscriber(); + source.subscribe(testSubscriber); + + testSubscriber.assertNoTerminalEvent(); + testSubscriber.assertNoValues(); + + testScheduler.triggerActions(); + + testSubscriber.awaitTerminalEvent(); + testSubscriber.assertNoErrors(); + testSubscriber.assertValue(soleValue); + } + + @Test(timeout = 60000) + public void testRequestNSingle() throws Exception { + final String[] values = {"Hello1", "Hello2"}; + Observable source = Observable.empty() + .concatEmptyWith(Observable.from(values)); + + TestSubscriber testSubscriber = new TestSubscriber(0); + source.subscribe(testSubscriber); + + testSubscriber.assertNoTerminalEvent(); + testSubscriber.assertNoErrors(); + testSubscriber.assertNoValues(); + + testSubscriber.requestMore(2); + + testSubscriber.awaitTerminalEvent(); + testSubscriber.assertNoErrors(); + testSubscriber.assertValues(values); + } + + @Test(timeout = 60000) + public void testRequestNMulti() throws Exception { + final String[] values = {"Hello1", "Hello2"}; + Observable source = Observable.empty() + .concatEmptyWith(Observable.from(values)); + + TestSubscriber testSubscriber = new TestSubscriber(0); + source.subscribe(testSubscriber); + + testSubscriber.assertNoTerminalEvent(); + testSubscriber.assertNoErrors(); + testSubscriber.assertNoValues(); + + testSubscriber.requestMore(1); + + testSubscriber.assertNoTerminalEvent(); + testSubscriber.assertNoErrors(); + testSubscriber.assertValues(values[0]); + + testSubscriber.requestMore(1); + + testSubscriber.awaitTerminalEvent(); + testSubscriber.assertNoErrors(); + testSubscriber.assertValues(values); + } + + @Test(timeout = 60000) + public void testSourceDontCompleteWithoutRequest() throws Exception { + + TestSubscriber testSubscriber = new TestSubscriber(0); + + String soleValue = "Hello"; + Observable.create(new OnSubscribe() { + @Override + public void call(final Subscriber subscriber) { + subscriber.setProducer(new Producer() { + @Override + public void request(long n) { + subscriber.onCompleted(); + } + }); + } + }).concatEmptyWith(Observable.just(soleValue)).subscribe(testSubscriber); + + testSubscriber.requestMore(1); + + testSubscriber.awaitTerminalEvent(); + testSubscriber.assertNoErrors(); + testSubscriber.assertValue(soleValue); + } +} diff --git a/src/test/java/rx/internal/operators/OperatorMergeEmptyWithTest.java b/src/test/java/rx/internal/operators/OperatorMergeEmptyWithTest.java new file mode 100644 index 0000000000..0bc0ea6a1f --- /dev/null +++ b/src/test/java/rx/internal/operators/OperatorMergeEmptyWithTest.java @@ -0,0 +1,145 @@ +/** + * Copyright 2014 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.internal.operators; + +import org.junit.Test; +import rx.Observable; +import rx.Observable.OnSubscribe; +import rx.Producer; +import rx.Subscriber; +import rx.observers.TestSubscriber; +import rx.schedulers.Schedulers; +import rx.schedulers.TestScheduler; + +import java.util.concurrent.TimeUnit; + +public class OperatorMergeEmptyWithTest { + + @Test(timeout = 60000) + public void testWithVoid() { + final String soleValue = "Hello"; + Observable source = Observable.empty() + .mergeEmptyWith(Observable.just(soleValue)); + + TestSubscriber testSubscriber = new TestSubscriber(); + source.subscribe(testSubscriber); + + testSubscriber.awaitTerminalEvent(); + testSubscriber.assertNoErrors(); + testSubscriber.assertValue(soleValue); + } + + @Test(timeout = 60000) + public void testErrorOnSourceEmitItem() { + TestScheduler testScheduler = Schedulers.test(); + Observable source = Observable.just(1) + .mergeEmptyWith(Observable.just("Hello").observeOn(testScheduler)); + + TestSubscriber testSubscriber = new TestSubscriber(); + source.subscribe(testSubscriber); + + testScheduler.triggerActions(); + testSubscriber.awaitTerminalEvent(); + testSubscriber.assertNoValues(); + testSubscriber.assertError(IllegalStateException.class); + } + + @Test(timeout = 60000) + public void testSourceError() throws Exception { + TestScheduler testScheduler = Schedulers.test(); + Observable source = Observable.error(new IllegalStateException()) + .mergeEmptyWith(Observable.just("Hello").observeOn(testScheduler)); + + TestSubscriber testSubscriber = new TestSubscriber(); + source.subscribe(testSubscriber); + + testScheduler.triggerActions(); + testSubscriber.awaitTerminalEvent(); + testSubscriber.assertNoValues(); + testSubscriber.assertError(IllegalStateException.class); + } + + @Test(timeout = 60000) + public void testSourceComplete() throws Exception { + final String soleValue = "Hello"; + Observable source = Observable.empty() + .mergeEmptyWith(Observable.just(soleValue)); + + TestSubscriber testSubscriber = new TestSubscriber(); + source.subscribe(testSubscriber); + + testSubscriber.awaitTerminalEvent(); + testSubscriber.assertNoErrors(); + testSubscriber.assertValue(soleValue); + } + + @Test(timeout = 60000) + public void testErrorFromSourcePostEmission() { + final String soleValue = "Hello"; + final TestScheduler testScheduler = Schedulers.test(); + + /*Delaying error event*/ + Observable source = Observable.error(new IllegalArgumentException()) + .observeOn(testScheduler) + .mergeEmptyWith(Observable.just(soleValue)); + + TestSubscriber testSubscriber = new TestSubscriber(); + source.subscribe(testSubscriber); + + testSubscriber.assertNotCompleted(); + testSubscriber.assertNoErrors(); + testSubscriber.assertValue(soleValue); + + testScheduler.advanceTimeBy(1, TimeUnit.HOURS); + testSubscriber.awaitTerminalEvent(); + testSubscriber.assertError(IllegalArgumentException.class); + } + + @Test(timeout = 60000) + public void testSourceNeverCompletes() throws Exception { + TestSubscriber subscriber = new TestSubscriber(); + Observable.never() + .mergeEmptyWith(Observable.just("Hello")) + .subscribe(subscriber); + + subscriber.assertValue("Hello"); + subscriber.assertNoTerminalEvent(); + } + + @Test(timeout = 60000) + public void testSourceDoesntCompleteWithoutRequest() throws Exception { + TestSubscriber testSubscriber = new TestSubscriber(0); + + String soleValue = "Hello"; + Observable.create(new OnSubscribe() { + @Override + public void call(final Subscriber subscriber) { + subscriber.setProducer(new Producer() { + @Override + public void request(long n) { + subscriber.onCompleted(); + } + }); + } + }).mergeEmptyWith(Observable.just(soleValue)).subscribe(testSubscriber); + + testSubscriber.requestMore(1); + + testSubscriber.awaitTerminalEvent(); + testSubscriber.assertNoErrors(); + testSubscriber.assertValue(soleValue); + } +} From d77a655923de156835d3e614f1eaf198957b51c5 Mon Sep 17 00:00:00 2001 From: George Campbell Date: Fri, 17 Jul 2015 16:27:48 -0700 Subject: [PATCH 02/81] Revert "No need to allocate a new head node." This reverts commit 46f9138f509f22be61d435cfb79335396fc92c48. --- .../rx/internal/operators/OperatorReplay.java | 28 ++++++++++--------- 1 file changed, 15 insertions(+), 13 deletions(-) diff --git a/src/main/java/rx/internal/operators/OperatorReplay.java b/src/main/java/rx/internal/operators/OperatorReplay.java index 2989f50b9e..77f19edf32 100644 --- a/src/main/java/rx/internal/operators/OperatorReplay.java +++ b/src/main/java/rx/internal/operators/OperatorReplay.java @@ -897,9 +897,9 @@ final void removeFirst() { size--; // can't just move the head because it would retain the very first value // can't null out the head's value because of late replayers would see null - setFirst(next); + setFirst(next.get()); } - /* test */ final void removeSome(int n) { + final void removeSome(int n) { Node head = get(); while (n > 0) { head = head.get(); @@ -907,14 +907,19 @@ final void removeFirst() { size--; } - setFirst(head); + setFirst(head.get()); } /** * Arranges the given node is the new head from now on. * @param n */ final void setFirst(Node n) { - set(n); + Node newHead = new Node(null); + newHead.lazySet(n); + if (n == null) { + tail = newHead; + } + set(newHead); } @Override @@ -1114,8 +1119,8 @@ Object leaveTransform(Object value) { void truncate() { long timeLimit = scheduler.now() - maxAgeInMillis; - Node prev = get(); - Node next = prev.get(); + Node head = get(); + Node next = head.get(); int e = 0; for (;;) { @@ -1123,14 +1128,12 @@ void truncate() { if (size > limit) { e++; size--; - prev = next; next = next.get(); } else { Timestamped v = (Timestamped)next.value; if (v.getTimestampMillis() <= timeLimit) { e++; size--; - prev = next; next = next.get(); } else { break; @@ -1141,15 +1144,15 @@ void truncate() { } } if (e != 0) { - setFirst(prev); + setFirst(next); } } @Override void truncateFinal() { long timeLimit = scheduler.now() - maxAgeInMillis; - Node prev = get(); - Node next = prev.get(); + Node head = get(); + Node next = head.get(); int e = 0; for (;;) { @@ -1158,7 +1161,6 @@ void truncateFinal() { if (v.getTimestampMillis() <= timeLimit) { e++; size--; - prev = next; next = next.get(); } else { break; @@ -1168,7 +1170,7 @@ void truncateFinal() { } } if (e != 0) { - setFirst(prev); + setFirst(next); } } } From fb2e540b58e073776a57caef75e711cdfb01242a Mon Sep 17 00:00:00 2001 From: George Campbell Date: Fri, 17 Jul 2015 16:27:55 -0700 Subject: [PATCH 03/81] Revert "Operator replay() now supports backpressure" This reverts commit 82d7b9cca2efd0a8f36ec3b700bb8f34c445a093. --- src/main/java/rx/Observable.java | 213 +-- .../rx/internal/operators/OperatorReplay.java | 1176 +---------------- .../operators/OperatorReplayTest.java | 160 +-- 3 files changed, 195 insertions(+), 1354 deletions(-) diff --git a/src/main/java/rx/Observable.java b/src/main/java/rx/Observable.java index 71313c657c..419591a8eb 100644 --- a/src/main/java/rx/Observable.java +++ b/src/main/java/rx/Observable.java @@ -25,6 +25,7 @@ import rx.observers.SafeSubscriber; import rx.plugins.*; import rx.schedulers.*; +import rx.subjects.*; import rx.subscriptions.Subscriptions; /** @@ -5978,9 +5979,9 @@ public Void call(Notification notification) { * *
*
Backpressure Support:
- *
This operator supports backpressure. Note that the upstream requests are determined by the child - * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will - * request 100 elements from the underlying Observable sequence.
+ *
This operator does not support backpressure because multicasting means the stream is "hot" with + * multiple subscribers. Each child will need to manage backpressure independently using operators such + * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
*
Scheduler:
*
This version of {@code replay} does not operate by default on a particular {@link Scheduler}.
*
@@ -5990,7 +5991,14 @@ public Void call(Notification notification) { * @see ReactiveX operators documentation: Replay */ public final ConnectableObservable replay() { - return OperatorReplay.create(this); + return new OperatorMulticast(this, new Func0>() { + + @Override + public Subject call() { + return ReplaySubject. create(); + } + + }); } /** @@ -6000,9 +6008,9 @@ public final ConnectableObservable replay() { * *
*
Backpressure Support:
- *
This operator supports backpressure. Note that the upstream requests are determined by the child - * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will - * request 100 elements from the underlying Observable sequence.
+ *
This operator does not support backpressure because multicasting means the stream is "hot" with + * multiple subscribers. Each child will need to manage backpressure independently using operators such + * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
*
Scheduler:
*
This version of {@code replay} does not operate by default on a particular {@link Scheduler}.
*
@@ -6017,12 +6025,12 @@ public final ConnectableObservable replay() { * @see ReactiveX operators documentation: Replay */ public final Observable replay(Func1, ? extends Observable> selector) { - return OperatorReplay.multicastSelector(new Func0>() { + return create(new OnSubscribeMulticastSelector(this, new Func0>() { @Override - public ConnectableObservable call() { - return Observable.this.replay(); + public final Subject call() { + return ReplaySubject.create(); } - }, selector); + }, selector)); } /** @@ -6033,9 +6041,9 @@ public ConnectableObservable call() { * *
*
Backpressure Support:
- *
This operator supports backpressure. Note that the upstream requests are determined by the child - * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will - * request 100 elements from the underlying Observable sequence.
+ *
This operator does not support backpressure because multicasting means the stream is "hot" with + * multiple subscribers. Each child will need to manage backpressure independently using operators such + * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
*
Scheduler:
*
This version of {@code replay} does not operate by default on a particular {@link Scheduler}.
*
@@ -6053,12 +6061,12 @@ public ConnectableObservable call() { * @see ReactiveX operators documentation: Replay */ public final Observable replay(Func1, ? extends Observable> selector, final int bufferSize) { - return OperatorReplay.multicastSelector(new Func0>() { + return create(new OnSubscribeMulticastSelector(this, new Func0>() { @Override - public ConnectableObservable call() { - return Observable.this.replay(bufferSize); + public final Subject call() { + return ReplaySubject.createWithSize(bufferSize); } - }, selector); + }, selector)); } /** @@ -6069,9 +6077,9 @@ public ConnectableObservable call() { * *
*
Backpressure Support:
- *
This operator supports backpressure. Note that the upstream requests are determined by the child - * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will - * request 100 elements from the underlying Observable sequence.
+ *
This operator does not support backpressure because multicasting means the stream is "hot" with + * multiple subscribers. Each child will need to manage backpressure independently using operators such + * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
*
Scheduler:
*
This version of {@code replay} operates by default on the {@code computation} {@link Scheduler}.
*
@@ -6105,9 +6113,9 @@ public final Observable replay(Func1, ? extends Obs * *
*
Backpressure Support:
- *
This operator supports backpressure. Note that the upstream requests are determined by the child - * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will - * request 100 elements from the underlying Observable sequence.
+ *
This operator does not support backpressure because multicasting means the stream is "hot" with + * multiple subscribers. Each child will need to manage backpressure independently using operators such + * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
*
Scheduler:
*
you specify which {@link Scheduler} this operator will use
*
@@ -6137,12 +6145,12 @@ public final Observable replay(Func1, ? extends Obs if (bufferSize < 0) { throw new IllegalArgumentException("bufferSize < 0"); } - return OperatorReplay.multicastSelector(new Func0>() { + return create(new OnSubscribeMulticastSelector(this, new Func0>() { @Override - public ConnectableObservable call() { - return Observable.this.replay(bufferSize, time, unit, scheduler); + public final Subject call() { + return ReplaySubject.createWithTimeAndSize(time, unit, bufferSize, scheduler); } - }, selector); + }, selector)); } /** @@ -6153,9 +6161,9 @@ public ConnectableObservable call() { * *
*
Backpressure Support:
- *
This operator supports backpressure. Note that the upstream requests are determined by the child - * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will - * request 100 elements from the underlying Observable sequence.
+ *
This operator does not support backpressure because multicasting means the stream is "hot" with + * multiple subscribers. Each child will need to manage backpressure independently using operators such + * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
*
Scheduler:
*
you specify which {@link Scheduler} this operator will use
*
@@ -6174,18 +6182,13 @@ public ConnectableObservable call() { * replaying no more than {@code bufferSize} notifications * @see ReactiveX operators documentation: Replay */ - public final Observable replay(final Func1, ? extends Observable> selector, final int bufferSize, final Scheduler scheduler) { - return OperatorReplay.multicastSelector(new Func0>() { + public final Observable replay(Func1, ? extends Observable> selector, final int bufferSize, final Scheduler scheduler) { + return create(new OnSubscribeMulticastSelector(this, new Func0>() { @Override - public ConnectableObservable call() { - return Observable.this.replay(bufferSize); + public final Subject call() { + return OperatorReplay. createScheduledSubject(ReplaySubject.createWithSize(bufferSize), scheduler); } - }, new Func1, Observable>() { - @Override - public Observable call(Observable t) { - return selector.call(t).observeOn(scheduler); - } - }); + }, selector)); } /** @@ -6196,9 +6199,9 @@ public Observable call(Observable t) { * *
*
Backpressure Support:
- *
This operator supports backpressure. Note that the upstream requests are determined by the child - * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will - * request 100 elements from the underlying Observable sequence.
+ *
This operator does not support backpressure because multicasting means the stream is "hot" with + * multiple subscribers. Each child will need to manage backpressure independently using operators such + * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
*
Scheduler:
*
This version of {@code replay} operates by default on the {@code computation} {@link Scheduler}.
*
@@ -6229,9 +6232,9 @@ public final Observable replay(Func1, ? extends Obs * *
*
Backpressure Support:
- *
This operator supports backpressure. Note that the upstream requests are determined by the child - * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will - * request 100 elements from the underlying Observable sequence.
+ *
This operator does not support backpressure because multicasting means the stream is "hot" with + * multiple subscribers. Each child will need to manage backpressure independently using operators such + * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
*
Scheduler:
*
you specify which {@link Scheduler} this operator will use
*
@@ -6253,12 +6256,12 @@ public final Observable replay(Func1, ? extends Obs * @see ReactiveX operators documentation: Replay */ public final Observable replay(Func1, ? extends Observable> selector, final long time, final TimeUnit unit, final Scheduler scheduler) { - return OperatorReplay.multicastSelector(new Func0>() { + return create(new OnSubscribeMulticastSelector(this, new Func0>() { @Override - public ConnectableObservable call() { - return Observable.this.replay(time, unit, scheduler); + public final Subject call() { + return ReplaySubject.createWithTime(time, unit, scheduler); } - }, selector); + }, selector)); } /** @@ -6268,9 +6271,9 @@ public ConnectableObservable call() { * *
*
Backpressure Support:
- *
This operator supports backpressure. Note that the upstream requests are determined by the child - * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will - * request 100 elements from the underlying Observable sequence.
+ *
This operator does not support backpressure because multicasting means the stream is "hot" with + * multiple subscribers. Each child will need to manage backpressure independently using operators such + * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
*
Scheduler:
*
you specify which {@link Scheduler} this operator will use
*
@@ -6287,18 +6290,13 @@ public ConnectableObservable call() { * replaying all items * @see ReactiveX operators documentation: Replay */ - public final Observable replay(final Func1, ? extends Observable> selector, final Scheduler scheduler) { - return OperatorReplay.multicastSelector(new Func0>() { + public final Observable replay(Func1, ? extends Observable> selector, final Scheduler scheduler) { + return create(new OnSubscribeMulticastSelector(this, new Func0>() { @Override - public ConnectableObservable call() { - return Observable.this.replay(); + public final Subject call() { + return OperatorReplay.createScheduledSubject(ReplaySubject. create(), scheduler); } - }, new Func1, Observable>() { - @Override - public Observable call(Observable t) { - return selector.call(t).observeOn(scheduler); - } - }); + }, selector)); } /** @@ -6310,9 +6308,9 @@ public Observable call(Observable t) { * *
*
Backpressure Support:
- *
This operator supports backpressure. Note that the upstream requests are determined by the child - * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will - * request 100 elements from the underlying Observable sequence.
+ *
This operator does not support backpressure because multicasting means the stream is "hot" with + * multiple subscribers. Each child will need to manage backpressure independently using operators such + * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
*
Scheduler:
*
This version of {@code replay} does not operate by default on a particular {@link Scheduler}.
*
@@ -6324,7 +6322,14 @@ public Observable call(Observable t) { * @see ReactiveX operators documentation: Replay */ public final ConnectableObservable replay(final int bufferSize) { - return OperatorReplay.create(this, bufferSize); + return new OperatorMulticast(this, new Func0>() { + + @Override + public Subject call() { + return ReplaySubject.createWithSize(bufferSize); + } + + }); } /** @@ -6336,9 +6341,9 @@ public final ConnectableObservable replay(final int bufferSize) { * *
*
Backpressure Support:
- *
This operator supports backpressure. Note that the upstream requests are determined by the child - * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will - * request 100 elements from the underlying Observable sequence.
+ *
This operator does not support backpressure because multicasting means the stream is "hot" with + * multiple subscribers. Each child will need to manage backpressure independently using operators such + * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
*
Scheduler:
*
This version of {@code replay} operates by default on the {@code computation} {@link Scheduler}.
*
@@ -6367,9 +6372,9 @@ public final ConnectableObservable replay(int bufferSize, long time, TimeUnit * *
*
Backpressure Support:
- *
This operator supports backpressure. Note that the upstream requests are determined by the child - * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will - * request 100 elements from the underlying Observable sequence.
+ *
This operator does not support backpressure because multicasting means the stream is "hot" with + * multiple subscribers. Each child will need to manage backpressure independently using operators such + * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
*
Scheduler:
*
you specify which {@link Scheduler} this operator will use
*
@@ -6393,7 +6398,14 @@ public final ConnectableObservable replay(final int bufferSize, final long ti if (bufferSize < 0) { throw new IllegalArgumentException("bufferSize < 0"); } - return OperatorReplay.create(this, time, unit, scheduler, bufferSize); + return new OperatorMulticast(this, new Func0>() { + + @Override + public Subject call() { + return ReplaySubject.createWithTimeAndSize(time, unit, bufferSize, scheduler); + } + + }); } /** @@ -6405,9 +6417,9 @@ public final ConnectableObservable replay(final int bufferSize, final long ti * *
*
Backpressure Support:
- *
This operator supports backpressure. Note that the upstream requests are determined by the child - * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will - * request 100 elements from the underlying Observable sequence.
+ *
This operator does not support backpressure because multicasting means the stream is "hot" with + * multiple subscribers. Each child will need to manage backpressure independently using operators such + * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
*
Scheduler:
*
you specify which {@link Scheduler} this operator will use
*
@@ -6421,7 +6433,14 @@ public final ConnectableObservable replay(final int bufferSize, final long ti * @see ReactiveX operators documentation: Replay */ public final ConnectableObservable replay(final int bufferSize, final Scheduler scheduler) { - return OperatorReplay.observeOn(replay(bufferSize), scheduler); + return new OperatorMulticast(this, new Func0>() { + + @Override + public Subject call() { + return OperatorReplay.createScheduledSubject(ReplaySubject.createWithSize(bufferSize), scheduler); + } + + }); } /** @@ -6433,9 +6452,9 @@ public final ConnectableObservable replay(final int bufferSize, final Schedul * *
*
Backpressure Support:
- *
This operator supports backpressure. Note that the upstream requests are determined by the child - * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will - * request 100 elements from the underlying Observable sequence.
+ *
This operator does not support backpressure because multicasting means the stream is "hot" with + * multiple subscribers. Each child will need to manage backpressure independently using operators such + * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
*
Scheduler:
*
This version of {@code replay} operates by default on the {@code computation} {@link Scheduler}.
*
@@ -6461,9 +6480,9 @@ public final ConnectableObservable replay(long time, TimeUnit unit) { * *
*
Backpressure Support:
- *
This operator supports backpressure. Note that the upstream requests are determined by the child - * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will - * request 100 elements from the underlying Observable sequence.
+ *
This operator does not support backpressure because multicasting means the stream is "hot" with + * multiple subscribers. Each child will need to manage backpressure independently using operators such + * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
*
Scheduler:
*
you specify which {@link Scheduler} this operator will use
*
@@ -6479,7 +6498,14 @@ public final ConnectableObservable replay(long time, TimeUnit unit) { * @see ReactiveX operators documentation: Replay */ public final ConnectableObservable replay(final long time, final TimeUnit unit, final Scheduler scheduler) { - return OperatorReplay.create(this, time, unit, scheduler); + return new OperatorMulticast(this, new Func0>() { + + @Override + public Subject call() { + return ReplaySubject.createWithTime(time, unit, scheduler); + } + + }); } /** @@ -6491,9 +6517,9 @@ public final ConnectableObservable replay(final long time, final TimeUnit uni * *
*
Backpressure Support:
- *
This operator supports backpressure. Note that the upstream requests are determined by the child - * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will - * request 100 elements from the underlying Observable sequence.
+ *
This operator does not support backpressure because multicasting means the stream is "hot" with + * multiple subscribers. Each child will need to manage backpressure independently using operators such + * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
*
Scheduler:
*
you specify which {@link Scheduler} this operator will use
*
@@ -6506,7 +6532,14 @@ public final ConnectableObservable replay(final long time, final TimeUnit uni * @see ReactiveX operators documentation: Replay */ public final ConnectableObservable replay(final Scheduler scheduler) { - return OperatorReplay.observeOn(replay(), scheduler); + return new OperatorMulticast(this, new Func0>() { + + @Override + public Subject call() { + return OperatorReplay.createScheduledSubject(ReplaySubject. create(), scheduler); + } + + }); } /** diff --git a/src/main/java/rx/internal/operators/OperatorReplay.java b/src/main/java/rx/internal/operators/OperatorReplay.java index 77f19edf32..83c76dfe39 100644 --- a/src/main/java/rx/internal/operators/OperatorReplay.java +++ b/src/main/java/rx/internal/operators/OperatorReplay.java @@ -15,1163 +15,93 @@ */ package rx.internal.operators; -import java.util.*; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.*; -import rx.*; import rx.Observable; -import rx.exceptions.Exceptions; -import rx.functions.*; -import rx.observables.ConnectableObservable; -import rx.schedulers.Timestamped; -import rx.subscriptions.Subscriptions; +import rx.Observable.OnSubscribe; +import rx.Scheduler; +import rx.Subscriber; +import rx.subjects.Subject; -public final class OperatorReplay extends ConnectableObservable { - /** The source observable. */ - final Observable source; - /** Holds the current subscriber that is, will be or just was subscribed to the source observable. */ - final AtomicReference> current; - /** A factory that creates the appropriate buffer for the ReplaySubscriber. */ - final Func0> bufferFactory; - - @SuppressWarnings("rawtypes") - static final Func0 DEFAULT_UNBOUNDED_FACTORY = new Func0() { - @Override - public Object call() { - return new UnboundedReplayBuffer(16); - } - }; - - /** - * Given a connectable observable factory, it multicasts over the generated - * ConnectableObservable via a selector function. - * @param connectableFactory - * @param selector - * @return - */ - public static Observable multicastSelector( - final Func0> connectableFactory, - final Func1, ? extends Observable> selector) { - return Observable.create(new OnSubscribe() { - @Override - public void call(final Subscriber child) { - ConnectableObservable co; - Observable observable; - try { - co = connectableFactory.call(); - observable = selector.call(co); - } catch (Throwable e) { - Exceptions.throwIfFatal(e); - child.onError(e); - return; - } - - observable.subscribe(child); - - co.connect(new Action1() { - @Override - public void call(Subscription t) { - child.add(t); - } - }); - } - }); - } - - /** - * Child Subscribers will observe the events of the ConnectableObservable on the - * specified scheduler. - * @param co - * @param scheduler - * @return - */ - public static ConnectableObservable observeOn(final ConnectableObservable co, final Scheduler scheduler) { - final Observable observable = co.observeOn(scheduler); - OnSubscribe onSubscribe = new OnSubscribe() { - @Override - public void call(final Subscriber child) { - // apply observeOn and prevent calling onStart() again - observable.unsafeSubscribe(new Subscriber(child) { - @Override - public void onNext(T t) { - child.onNext(t); - } - @Override - public void onError(Throwable e) { - child.onError(e); - } - @Override - public void onCompleted() { - child.onCompleted(); - } - }); - } - }; - return new ConnectableObservable(onSubscribe) { - @Override - public void connect(Action1 connection) { - co.connect(connection); - } - }; - } - - /** - * Creates a replaying ConnectableObservable with an unbounded buffer. - * @param source - * @return - */ - @SuppressWarnings("unchecked") - public static ConnectableObservable create(Observable source) { - return create(source, DEFAULT_UNBOUNDED_FACTORY); - } - - /** - * Creates a replaying ConnectableObservable with a size bound buffer. - * @param source - * @param bufferSize - * @return - */ - public static ConnectableObservable create(Observable source, - final int bufferSize) { - if (bufferSize == Integer.MAX_VALUE) { - return create(source); - } - return create(source, new Func0>() { - @Override - public ReplayBuffer call() { - return new SizeBoundReplayBuffer(bufferSize); - } - }); +/** + * Replay with limited buffer and/or time constraints. + * + * + * @see MSDN: Observable.Replay overloads + */ +public final class OperatorReplay { + /** Utility class. */ + private OperatorReplay() { + throw new IllegalStateException("No instances!"); } /** - * Creates a replaying ConnectableObservable with a time bound buffer. - * @param source - * @param maxAge - * @param unit - * @param scheduler - * @return + * Creates a subject whose client observers will observe events + * propagated through the given wrapped subject. + * @param the element type + * @param subject the subject to wrap + * @param scheduler the target scheduler + * @return the created subject */ - public static ConnectableObservable create(Observable source, - long maxAge, TimeUnit unit, Scheduler scheduler) { - return create(source, maxAge, unit, scheduler, Integer.MAX_VALUE); - } + public static Subject createScheduledSubject(Subject subject, Scheduler scheduler) { + final Observable observedOn = subject.observeOn(scheduler); + SubjectWrapper s = new SubjectWrapper(new OnSubscribe() { - /** - * Creates a replaying ConnectableObservable with a size and time bound buffer. - * @param source - * @param maxAge - * @param unit - * @param scheduler - * @param bufferSize - * @return - */ - public static ConnectableObservable create(Observable source, - long maxAge, TimeUnit unit, final Scheduler scheduler, final int bufferSize) { - final long maxAgeInMillis = unit.toMillis(maxAge); - return create(source, new Func0>() { @Override - public ReplayBuffer call() { - return new SizeAndTimeBoundReplayBuffer(bufferSize, maxAgeInMillis, scheduler); + public void call(Subscriber o) { + subscriberOf(observedOn).call(o); } - }); + + }, subject); + return s; } /** - * Creates a OperatorReplay instance to replay values of the given source observable. - * @param source the source observable - * @param bufferFactory the factory to instantiate the appropriate buffer when the observable becomes active - * @return the connectable observable + * Return an OnSubscribeFunc which delegates the subscription to the given observable. + * + * @param the value type + * @param target the target observable + * @return the function that delegates the subscription to the target */ - static ConnectableObservable create(Observable source, - final Func0> bufferFactory) { - // the current connection to source needs to be shared between the operator and its onSubscribe call - final AtomicReference> curr = new AtomicReference>(); - OnSubscribe onSubscribe = new OnSubscribe() { + public static OnSubscribe subscriberOf(final Observable target) { + return new OnSubscribe() { @Override - public void call(Subscriber child) { - // concurrent connection/disconnection may change the state, - // we loop to be atomic while the child subscribes - for (;;) { - // get the current subscriber-to-source - ReplaySubscriber r = curr.get(); - // if there isn't one - if (r == null) { - // create a new subscriber to source - ReplaySubscriber u = new ReplaySubscriber(curr, bufferFactory.call()); - // perform extra initialization to avoid 'this' to escape during construction - u.init(); - // let's try setting it as the current subscriber-to-source - if (!curr.compareAndSet(r, u)) { - // didn't work, maybe someone else did it or the current subscriber - // to source has just finished - continue; - } - // we won, let's use it going onwards - r = u; - } - - // create the backpressure-managing producer for this child - InnerProducer inner = new InnerProducer(r, child); - // we try to add it to the array of producers - // if it fails, no worries because we will still have its buffer - // so it is going to replay it for us - r.add(inner); - // the producer has been registered with the current subscriber-to-source so - // at least it will receive the next terminal event - child.add(inner); - // setting the producer will trigger the first request to be considered by - // the subscriber-to-source. - child.setProducer(inner); - break; - } + public void call(Subscriber t1) { + target.unsafeSubscribe(t1); } }; - return new OperatorReplay(onSubscribe, source, curr, bufferFactory); - } - private OperatorReplay(OnSubscribe onSubscribe, Observable source, - final AtomicReference> current, - final Func0> bufferFactory) { - super(onSubscribe); - this.source = source; - this.current = current; - this.bufferFactory = bufferFactory; - } - - @Override - public void connect(Action1 connection) { - boolean doConnect = false; - ReplaySubscriber ps; - // we loop because concurrent connect/disconnect and termination may change the state - for (;;) { - // retrieve the current subscriber-to-source instance - ps = current.get(); - // if there is none yet or the current has unsubscribed - if (ps == null || ps.isUnsubscribed()) { - // create a new subscriber-to-source - ReplaySubscriber u = new ReplaySubscriber(current, bufferFactory.call()); - // initialize out the constructor to avoid 'this' to escape - u.init(); - // try setting it as the current subscriber-to-source - if (!current.compareAndSet(ps, u)) { - // did not work, perhaps a new subscriber arrived - // and created a new subscriber-to-source as well, retry - continue; - } - ps = u; - } - // if connect() was called concurrently, only one of them should actually - // connect to the source - doConnect = !ps.shouldConnect.get() && ps.shouldConnect.compareAndSet(false, true); - break; - } - /* - * Notify the callback that we have a (new) connection which it can unsubscribe - * but since ps is unique to a connection, multiple calls to connect() will return the - * same Subscription and even if there was a connect-disconnect-connect pair, the older - * references won't disconnect the newer connection. - * Synchronous source consumers have the opportunity to disconnect via unsubscribe on the - * Subscription as unsafeSubscribe may never return in its own. - * - * Note however, that asynchronously disconnecting a running source might leave - * child-subscribers without any terminal event; ReplaySubject does not have this - * issue because the unsubscription was always triggered by the child-subscribers - * themselves. - */ - connection.call(ps); - if (doConnect) { - source.unsafeSubscribe(ps); - } } - - @SuppressWarnings("rawtypes") - static final class ReplaySubscriber extends Subscriber implements Subscription { - /** Holds notifications from upstream. */ - final ReplayBuffer buffer; - /** The notification-lite factory. */ - final NotificationLite nl; - /** Contains either an onCompleted or an onError token from upstream. */ - boolean done; - - /** Indicates an empty array of inner producers. */ - static final InnerProducer[] EMPTY = new InnerProducer[0]; - /** Indicates a terminated ReplaySubscriber. */ - static final InnerProducer[] TERMINATED = new InnerProducer[0]; - - /** Tracks the subscribed producers. */ - final AtomicReference producers; - /** - * Atomically changed from false to true by connect to make sure the - * connection is only performed by one thread. - */ - final AtomicBoolean shouldConnect; - - /** Guarded by this. */ - boolean emitting; - /** Guarded by this. */ - boolean missed; - - - /** Contains the maximum element index the child Subscribers requested so far. Accessed while emitting is true. */ - long maxChildRequested; - /** Counts the outstanding upstream requests until the producer arrives. */ - long maxUpstreamRequested; - /** The upstream producer. */ - volatile Producer producer; - - public ReplaySubscriber(AtomicReference> current, - ReplayBuffer buffer) { - this.buffer = buffer; - - this.nl = NotificationLite.instance(); - this.producers = new AtomicReference(EMPTY); - this.shouldConnect = new AtomicBoolean(); - // make sure the source doesn't produce values until the child subscribers - // expressed their request amounts - this.request(0); - } - /** Should be called after the constructor finished to setup nulling-out the current reference. */ - void init() { - add(Subscriptions.create(new Action0() { - @Override - public void call() { - ReplaySubscriber.this.producers.getAndSet(TERMINATED); - // unlike OperatorPublish, we can't null out the terminated so - // late subscribers can still get replay - // current.compareAndSet(ReplaySubscriber.this, null); - // we don't care if it fails because it means the current has - // been replaced in the meantime - } - })); - } - /** - * Atomically try adding a new InnerProducer to this Subscriber or return false if this - * Subscriber was terminated. - * @param producer the producer to add - * @return true if succeeded, false otherwise - */ - boolean add(InnerProducer producer) { - if (producer == null) { - throw new NullPointerException(); - } - // the state can change so we do a CAS loop to achieve atomicity - for (;;) { - // get the current producer array - InnerProducer[] c = producers.get(); - // if this subscriber-to-source reached a terminal state by receiving - // an onError or onCompleted, just refuse to add the new producer - if (c == TERMINATED) { - return false; - } - // we perform a copy-on-write logic - int len = c.length; - InnerProducer[] u = new InnerProducer[len + 1]; - System.arraycopy(c, 0, u, 0, len); - u[len] = producer; - // try setting the producers array - if (producers.compareAndSet(c, u)) { - return true; - } - // if failed, some other operation succeded (another add, remove or termination) - // so retry - } - } - - /** - * Atomically removes the given producer from the producers array. - * @param producer the producer to remove - */ - void remove(InnerProducer producer) { - // the state can change so we do a CAS loop to achieve atomicity - for (;;) { - // let's read the current producers array - InnerProducer[] c = producers.get(); - // if it is either empty or terminated, there is nothing to remove so we quit - if (c == EMPTY || c == TERMINATED) { - return; - } - // let's find the supplied producer in the array - // although this is O(n), we don't expect too many child subscribers in general - int j = -1; - int len = c.length; - for (int i = 0; i < len; i++) { - if (c[i].equals(producer)) { - j = i; - break; - } - } - // we didn't find it so just quit - if (j < 0) { - return; - } - // we do copy-on-write logic here - InnerProducer[] u; - // we don't create a new empty array if producer was the single inhabitant - // but rather reuse an empty array - if (len == 1) { - u = EMPTY; - } else { - // otherwise, create a new array one less in size - u = new InnerProducer[len - 1]; - // copy elements being before the given producer - System.arraycopy(c, 0, u, 0, j); - // copy elements being after the given producer - System.arraycopy(c, j + 1, u, j, len - j - 1); - } - // try setting this new array as - if (producers.compareAndSet(c, u)) { - return; - } - // if we failed, it means something else happened - // (a concurrent add/remove or termination), we need to retry - } - } - - @Override - public void setProducer(Producer p) { - Producer p0 = producer; - if (p0 != null) { - throw new IllegalStateException("Only a single producer can be set on a Subscriber."); - } - producer = p; - manageRequests(); - replay(); - } - - @Override - public void onNext(T t) { - if (!done) { - buffer.next(t); - replay(); - } - } - @Override - public void onError(Throwable e) { - // The observer front is accessed serially as required by spec so - // no need to CAS in the terminal value - if (!done) { - done = true; - try { - buffer.error(e); - replay(); - } finally { - unsubscribe(); // expectation of testIssue2191 - } - } - } - @Override - public void onCompleted() { - // The observer front is accessed serially as required by spec so - // no need to CAS in the terminal value - if (!done) { - done = true; - try { - buffer.complete(); - replay(); - } finally { - unsubscribe(); - } - } - } - - /** - * Coordinates the request amounts of various child Subscribers. - */ - void manageRequests() { - // if the upstream has completed, no more requesting is possible - if (isUnsubscribed()) { - return; - } - synchronized (this) { - if (emitting) { - missed = true; - return; - } - emitting = true; - } - for (;;) { - // if the upstream has completed, no more requesting is possible - if (isUnsubscribed()) { - return; - } - - @SuppressWarnings("unchecked") - InnerProducer[] a = producers.get(); - - long ri = maxChildRequested; - long maxTotalRequests = 0; - - for (InnerProducer rp : a) { - maxTotalRequests = Math.max(maxTotalRequests, rp.totalRequested.get()); - } - - long ur = maxUpstreamRequested; - Producer p = producer; - long diff = maxTotalRequests - ri; - if (diff != 0) { - maxChildRequested = maxTotalRequests; - if (p != null) { - if (ur != 0L) { - maxUpstreamRequested = 0L; - p.request(ur + diff); - } else { - p.request(diff); - } - } else { - // collect upstream request amounts until there is a producer for them - long u = ur + diff; - if (u < 0) { - u = Long.MAX_VALUE; - } - maxUpstreamRequested = u; - } - } else - // if there were outstanding upstream requests and we have a producer - if (ur != 0L && p != null) { - maxUpstreamRequested = 0L; - // fire the accumulated requests - p.request(ur); - } - - synchronized (this) { - if (!missed) { - emitting = false; - return; - } - missed = false; - } - } - } - - /** - * Tries to replay the buffer contents to all known subscribers. - */ - void replay() { - @SuppressWarnings("unchecked") - InnerProducer[] a = producers.get(); - for (InnerProducer rp : a) { - buffer.replay(rp); - } - } - } - /** - * A Producer and Subscription that manages the request and unsubscription state of a - * child subscriber in thread-safe manner. - * We use AtomicLong as a base class to save on extra allocation of an AtomicLong and also - * save the overhead of the AtomicIntegerFieldUpdater. - * @param the value type - */ - static final class InnerProducer extends AtomicLong implements Producer, Subscription { - /** */ - private static final long serialVersionUID = -4453897557930727610L; - /** - * The parent subscriber-to-source used to allow removing the child in case of - * child unsubscription. - */ - final ReplaySubscriber parent; - /** The actual child subscriber. */ - final Subscriber child; - /** - * Holds an object that represents the current location in the buffer. - * Guarded by the emitter loop. - */ - Object index; - /** - * Keeps the sum of all requested amounts. - */ - final AtomicLong totalRequested; - /** Indicates an emission state. Guarded by this. */ - boolean emitting; - /** Indicates a missed update. Guarded by this. */ - boolean missed; - /** - * Indicates this child has been unsubscribed: the state is swapped in atomically and - * will prevent the dispatch() to emit (too many) values to a terminated child subscriber. - */ - static final long UNSUBSCRIBED = Long.MIN_VALUE; - - public InnerProducer(ReplaySubscriber parent, Subscriber child) { - this.parent = parent; - this.child = child; - this.totalRequested = new AtomicLong(); - } - - @Override - public void request(long n) { - // ignore negative requests - if (n < 0) { - return; - } - // In general, RxJava doesn't prevent concurrent requests (with each other or with - // an unsubscribe) so we need a CAS-loop, but we need to handle - // request overflow and unsubscribed/not requested state as well. - for (;;) { - // get the current request amount - long r = get(); - // if child called unsubscribe() do nothing - if (r == UNSUBSCRIBED) { - return; - } - // ignore zero requests except any first that sets in zero - if (r >= 0L && n == 0) { - return; - } - // otherwise, increase the request count - long u = r + n; - // and check for long overflow - if (u < 0) { - // cap at max value, which is essentially unlimited - u = Long.MAX_VALUE; - } - // try setting the new request value - if (compareAndSet(r, u)) { - // increment the total request counter - addTotalRequested(n); - // if successful, notify the parent dispacher this child can receive more - // elements - parent.manageRequests(); - - parent.buffer.replay(this); - return; - } - // otherwise, someone else changed the state (perhaps a concurrent - // request or unsubscription so retry - } - } - - /** - * Increments the total requested amount. - * @param n the additional request amount - */ - void addTotalRequested(long n) { - for (;;) { - long r = totalRequested.get(); - long u = r + n; - if (u < 0) { - u = Long.MAX_VALUE; - } - if (totalRequested.compareAndSet(r, u)) { - return; - } - } - } - - /** - * Indicate that values have been emitted to this child subscriber by the dispatch() method. - * @param n the number of items emitted - * @return the updated request value (may indicate how much can be produced or a terminal state) - */ - public long produced(long n) { - // we don't allow producing zero or less: it would be a bug in the operator - if (n <= 0) { - throw new IllegalArgumentException("Cant produce zero or less"); - } - for (;;) { - // get the current request value - long r = get(); - // if the child has unsubscribed, simply return and indicate this - if (r == UNSUBSCRIBED) { - return UNSUBSCRIBED; - } - // reduce the requested amount - long u = r - n; - // if the new amount is less than zero, we have a bug in this operator - if (u < 0) { - throw new IllegalStateException("More produced (" + n + ") than requested (" + r + ")"); - } - // try updating the request value - if (compareAndSet(r, u)) { - // and return the udpated value - return u; - } - // otherwise, some concurrent activity happened and we need to retry - } - } - - @Override - public boolean isUnsubscribed() { - return get() == UNSUBSCRIBED; - } - @Override - public void unsubscribe() { - long r = get(); - // let's see if we are unsubscribed - if (r != UNSUBSCRIBED) { - // if not, swap in the terminal state, this is idempotent - // because other methods using CAS won't overwrite this value, - // concurrent calls to unsubscribe will atomically swap in the same - // terminal value - r = getAndSet(UNSUBSCRIBED); - // and only one of them will see a non-terminated value before the swap - if (r != UNSUBSCRIBED) { - // remove this from the parent - parent.remove(this); - // After removal, we might have unblocked the other child subscribers: - // let's assume this child had 0 requested before the unsubscription while - // the others had non-zero. By removing this 'blocking' child, the others - // are now free to receive events - parent.manageRequests(); - } - } - } - /** - * Convenience method to auto-cast the index object. - * @return - */ - @SuppressWarnings("unchecked") - U index() { - return (U)index; - } - } /** - * The interface for interacting with various buffering logic. - * + * A subject that wraps another subject. * @param the value type */ - interface ReplayBuffer { - /** - * Adds a regular value to the buffer. - * @param value - */ - void next(T value); - /** - * Adds a terminal exception to the buffer - * @param e - */ - void error(Throwable e); - /** - * Adds a completion event to the buffer - */ - void complete(); - /** - * Tries to replay the buffered values to the - * subscriber inside the output if there - * is new value and requests available at the - * same time. - * @param output - */ - void replay(InnerProducer output); - } - - /** - * Holds an unbounded list of events. - * - * @param the value type - */ - static final class UnboundedReplayBuffer extends ArrayList implements ReplayBuffer { - /** */ - private static final long serialVersionUID = 7063189396499112664L; - final NotificationLite nl; - /** The total number of events in the buffer. */ - volatile int size; - - public UnboundedReplayBuffer(int capacityHint) { - super(capacityHint); - nl = NotificationLite.instance(); - } - @Override - public void next(T value) { - add(nl.next(value)); - size++; - } - - @Override - public void error(Throwable e) { - add(nl.error(e)); - size++; - } + public static final class SubjectWrapper extends Subject { + /** The wrapped subject. */ + final Subject subject; - @Override - public void complete() { - add(nl.completed()); - size++; + public SubjectWrapper(OnSubscribe func, Subject subject) { + super(func); + this.subject = subject; } @Override - public void replay(InnerProducer output) { - synchronized (output) { - if (output.emitting) { - output.missed = true; - return; - } - output.emitting = true; - } - for (;;) { - if (output.isUnsubscribed()) { - return; - } - int sourceIndex = size; - - Integer destIndexObject = output.index(); - int destIndex = destIndexObject != null ? destIndexObject.intValue() : 0; - - long r = output.get(); - long r0 = r; - long e = 0L; - - while (r != 0L && destIndex < sourceIndex) { - Object o = get(destIndex); - if (nl.accept(output.child, o)) { - return; - } - if (output.isUnsubscribed()) { - return; - } - destIndex++; - r--; - e++; - } - if (e != 0L) { - output.index = destIndex; - if (r0 != Long.MAX_VALUE) { - output.produced(e); - } - } - - synchronized (output) { - if (!output.missed) { - output.emitting = false; - return; - } - output.missed = false; - } - } - } - } - - /** - * Represents a node in a bounded replay buffer's linked list. - * - * @param the contained value type - */ - static final class Node extends AtomicReference { - /** */ - private static final long serialVersionUID = 245354315435971818L; - final Object value; - public Node(Object value) { - this.value = value; - } - } - - /** - * Base class for bounded buffering with options to specify an - * enter and leave transforms and custom truncation behavior. - * - * @param the value type - */ - static class BoundedReplayBuffer extends AtomicReference implements ReplayBuffer { - /** */ - private static final long serialVersionUID = 2346567790059478686L; - final NotificationLite nl; - - Node tail; - int size; - - public BoundedReplayBuffer() { - nl = NotificationLite.instance(); - Node n = new Node(null); - tail = n; - set(n); - } - - /** - * Add a new node to the linked list. - * @param n - */ - final void addLast(Node n) { - tail.set(n); - tail = n; - size++; - } - /** - * Remove the first node from the linked list. - */ - final void removeFirst() { - Node head = get(); - Node next = head.get(); - if (next == null) { - throw new IllegalStateException("Empty list!"); - } - size--; - // can't just move the head because it would retain the very first value - // can't null out the head's value because of late replayers would see null - setFirst(next.get()); - } - final void removeSome(int n) { - Node head = get(); - while (n > 0) { - head = head.get(); - n--; - size--; - } - - setFirst(head.get()); - } - /** - * Arranges the given node is the new head from now on. - * @param n - */ - final void setFirst(Node n) { - Node newHead = new Node(null); - newHead.lazySet(n); - if (n == null) { - tail = newHead; - } - set(newHead); - } - - @Override - public final void next(T value) { - Object o = enterTransform(nl.next(value)); - Node n = new Node(o); - addLast(n); - truncate(); + public void onNext(T args) { + subject.onNext(args); } @Override - public final void error(Throwable e) { - Object o = enterTransform(nl.error(e)); - Node n = new Node(o); - addLast(n); - truncateFinal(); + public void onError(Throwable e) { + subject.onError(e); } @Override - public final void complete() { - Object o = enterTransform(nl.completed()); - Node n = new Node(o); - addLast(n); - truncateFinal(); + public void onCompleted() { + subject.onCompleted(); } @Override - public final void replay(InnerProducer output) { - synchronized (output) { - if (output.emitting) { - output.missed = true; - return; - } - output.emitting = true; - } - for (;;) { - if (output.isUnsubscribed()) { - return; - } - - long r = output.get(); - long r0 = r; - long e = 0L; - - Node node = output.index(); - if (node == null) { - node = get(); - output.index = node; - } - - while (r != 0) { - Node v = node.get(); - if (v != null) { - Object o = leaveTransform(v.value); - if (nl.accept(output.child, o)) { - output.index = null; - return; - } - e++; - node = v; - } else { - break; - } - if (output.isUnsubscribed()) { - return; - } - } - - if (e != 0L) { - output.index = node; - if (r0 != Long.MAX_VALUE) { - output.produced(e); - } - } - - synchronized (output) { - if (!output.missed) { - output.emitting = false; - return; - } - output.missed = false; - } - } - - } - - /** - * Override this to wrap the NotificationLite object into a - * container to be used later by truncate. - * @param value - * @return - */ - Object enterTransform(Object value) { - return value; - } - /** - * Override this to unwrap the transformed value into a - * NotificationLite object. - * @param value - * @return - */ - Object leaveTransform(Object value) { - return value; - } - /** - * Override this method to truncate a non-terminated buffer - * based on its current properties. - */ - void truncate() { - - } - /** - * Override this method to truncate a terminated buffer - * based on its properties (i.e., truncate but the very last node). - */ - void truncateFinal() { - - } - /* test */ final void collect(Collection output) { - Node n = get(); - for (;;) { - Node next = n.get(); - if (next != null) { - Object o = next.value; - Object v = leaveTransform(o); - if (nl.isCompleted(v) || nl.isError(v)) { - break; - } - output.add(nl.getValue(v)); - n = next; - } else { - break; - } - } - } - /* test */ boolean hasError() { - return tail.value != null && nl.isError(leaveTransform(tail.value)); - } - /* test */ boolean hasCompleted() { - return tail.value != null && nl.isCompleted(leaveTransform(tail.value)); - } - } - - /** - * A bounded replay buffer implementation with size limit only. - * - * @param the value type - */ - static final class SizeBoundReplayBuffer extends BoundedReplayBuffer { - /** */ - private static final long serialVersionUID = -5898283885385201806L; - - final int limit; - public SizeBoundReplayBuffer(int limit) { - this.limit = limit; - } - - @Override - void truncate() { - // overflow can be at most one element - if (size > limit) { - removeFirst(); - } - } - - // no need for final truncation because values are truncated one by one - } - - /** - * Size and time bound replay buffer. - * - * @param the buffered value type - */ - static final class SizeAndTimeBoundReplayBuffer extends BoundedReplayBuffer { - /** */ - private static final long serialVersionUID = 3457957419649567404L; - final Scheduler scheduler; - final long maxAgeInMillis; - final int limit; - public SizeAndTimeBoundReplayBuffer(int limit, long maxAgeInMillis, Scheduler scheduler) { - this.scheduler = scheduler; - this.limit = limit; - this.maxAgeInMillis = maxAgeInMillis; - } - - @Override - Object enterTransform(Object value) { - return new Timestamped(scheduler.now(), value); - } - - @Override - Object leaveTransform(Object value) { - return ((Timestamped)value).getValue(); - } - - @Override - void truncate() { - long timeLimit = scheduler.now() - maxAgeInMillis; - - Node head = get(); - Node next = head.get(); - - int e = 0; - for (;;) { - if (next != null) { - if (size > limit) { - e++; - size--; - next = next.get(); - } else { - Timestamped v = (Timestamped)next.value; - if (v.getTimestampMillis() <= timeLimit) { - e++; - size--; - next = next.get(); - } else { - break; - } - } - } else { - break; - } - } - if (e != 0) { - setFirst(next); - } - } - @Override - void truncateFinal() { - long timeLimit = scheduler.now() - maxAgeInMillis; - - Node head = get(); - Node next = head.get(); - - int e = 0; - for (;;) { - if (next != null && size > 1) { - Timestamped v = (Timestamped)next.value; - if (v.getTimestampMillis() <= timeLimit) { - e++; - size--; - next = next.get(); - } else { - break; - } - } else { - break; - } - } - if (e != 0) { - setFirst(next); - } + public boolean hasObservers() { + return this.subject.hasObservers(); } } -} +} \ No newline at end of file diff --git a/src/test/java/rx/internal/operators/OperatorReplayTest.java b/src/test/java/rx/internal/operators/OperatorReplayTest.java index 5c31503da4..a5ff85864d 100644 --- a/src/test/java/rx/internal/operators/OperatorReplayTest.java +++ b/src/test/java/rx/internal/operators/OperatorReplayTest.java @@ -16,27 +16,33 @@ package rx.internal.operators; import static org.junit.Assert.assertEquals; -import static org.mockito.Matchers.*; -import static org.mockito.Mockito.*; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.notNull; +import static org.mockito.Mockito.inOrder; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; -import java.util.*; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.*; +import java.util.concurrent.atomic.AtomicInteger; -import org.junit.*; +import org.junit.Test; import org.mockito.InOrder; -import rx.*; -import rx.Scheduler.Worker; import rx.Observable; import rx.Observer; -import rx.functions.*; -import rx.internal.operators.OperatorReplay.BoundedReplayBuffer; -import rx.internal.operators.OperatorReplay.Node; -import rx.internal.operators.OperatorReplay.SizeAndTimeBoundReplayBuffer; +import rx.Scheduler; +import rx.Scheduler.Worker; +import rx.Subscription; +import rx.functions.Action0; +import rx.functions.Action1; +import rx.functions.Func1; import rx.observables.ConnectableObservable; -import rx.observers.TestSubscriber; -import rx.schedulers.*; +import rx.schedulers.TestScheduler; import rx.subjects.PublishSubject; public class OperatorReplayTest { @@ -733,132 +739,4 @@ public boolean isUnsubscribed() { } } - @Test - public void testBoundedReplayBuffer() { - BoundedReplayBuffer buf = new BoundedReplayBuffer(); - buf.addLast(new Node(1)); - buf.addLast(new Node(2)); - buf.addLast(new Node(3)); - buf.addLast(new Node(4)); - buf.addLast(new Node(5)); - - List values = new ArrayList(); - buf.collect(values); - - Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5), values); - - buf.removeSome(2); - buf.removeFirst(); - buf.removeSome(2); - - values.clear(); - buf.collect(values); - Assert.assertTrue(values.isEmpty()); - - buf.addLast(new Node(5)); - buf.addLast(new Node(6)); - buf.collect(values); - - Assert.assertEquals(Arrays.asList(5, 6), values); - - } - - @Test - public void testTimedAndSizedTruncation() { - TestScheduler test = Schedulers.test(); - SizeAndTimeBoundReplayBuffer buf = new SizeAndTimeBoundReplayBuffer(2, 2000, test); - List values = new ArrayList(); - - buf.next(1); - test.advanceTimeBy(1, TimeUnit.SECONDS); - buf.next(2); - test.advanceTimeBy(1, TimeUnit.SECONDS); - buf.collect(values); - Assert.assertEquals(Arrays.asList(1, 2), values); - - buf.next(3); - buf.next(4); - values.clear(); - buf.collect(values); - Assert.assertEquals(Arrays.asList(3, 4), values); - - test.advanceTimeBy(2, TimeUnit.SECONDS); - buf.next(5); - - values.clear(); - buf.collect(values); - Assert.assertEquals(Arrays.asList(5), values); - - test.advanceTimeBy(2, TimeUnit.SECONDS); - buf.complete(); - - values.clear(); - buf.collect(values); - Assert.assertTrue(values.isEmpty()); - - Assert.assertEquals(1, buf.size); - Assert.assertTrue(buf.hasCompleted()); - } - - @Test - public void testBackpressure() { - final AtomicLong requested = new AtomicLong(); - Observable source = Observable.range(1, 1000) - .doOnRequest(new Action1() { - @Override - public void call(Long t) { - requested.addAndGet(t); - } - }); - ConnectableObservable co = source.replay(); - - TestSubscriber ts1 = TestSubscriber.create(10); - TestSubscriber ts2 = TestSubscriber.create(90); - - co.subscribe(ts1); - co.subscribe(ts2); - - ts2.requestMore(10); - - co.connect(); - - ts1.assertValueCount(10); - ts1.assertNoTerminalEvent(); - - ts2.assertValueCount(100); - ts2.assertNoTerminalEvent(); - - Assert.assertEquals(100, requested.get()); - } - - @Test - public void testBackpressureBounded() { - final AtomicLong requested = new AtomicLong(); - Observable source = Observable.range(1, 1000) - .doOnRequest(new Action1() { - @Override - public void call(Long t) { - requested.addAndGet(t); - } - }); - ConnectableObservable co = source.replay(50); - - TestSubscriber ts1 = TestSubscriber.create(10); - TestSubscriber ts2 = TestSubscriber.create(90); - - co.subscribe(ts1); - co.subscribe(ts2); - - ts2.requestMore(10); - - co.connect(); - - ts1.assertValueCount(10); - ts1.assertNoTerminalEvent(); - - ts2.assertValueCount(100); - ts2.assertNoTerminalEvent(); - - Assert.assertEquals(100, requested.get()); - } } \ No newline at end of file From 09957fb5af53f182002d92c0497a7159821a20a0 Mon Sep 17 00:00:00 2001 From: George Campbell Date: Fri, 17 Jul 2015 16:33:59 -0700 Subject: [PATCH 04/81] Revert "If cache() now supports backpressure, correct javadocs to indicate this." This reverts commit ec3d522c826c3135b9f5e3a9bb34f62756ec95cc. --- src/main/java/rx/Observable.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/main/java/rx/Observable.java b/src/main/java/rx/Observable.java index 419591a8eb..60d814186f 100644 --- a/src/main/java/rx/Observable.java +++ b/src/main/java/rx/Observable.java @@ -3583,7 +3583,8 @@ public final Observable> buffer(Observable boundary, int initialC * of items that will use up memory. *
*
Backpressure Support:
- *
This operator supports backpressure.
+ *
This operator does not support upstream backpressure as it is purposefully requesting and caching + * everything emitted.
*
Scheduler:
*
{@code cache} does not operate by default on a particular {@link Scheduler}.
*
@@ -3616,7 +3617,8 @@ public final Observable cache() { * of items that will use up memory. *
*
Backpressure Support:
- *
This operator supports backpressure.
+ *
This operator does not support upstream backpressure as it is purposefully requesting and caching + * everything emitted.
*
Scheduler:
*
{@code cache} does not operate by default on a particular {@link Scheduler}.
*
From a641321e3cc6925385d0fcfdb2c939ac3fce7dce Mon Sep 17 00:00:00 2001 From: George Campbell Date: Fri, 17 Jul 2015 16:39:26 -0700 Subject: [PATCH 05/81] Revert "cache now supports backpressure" This reverts commit 18ff5afd380625f9157d9e9a3144baf845c09086. --- src/main/java/rx/Observable.java | 4 +- .../internal/operators/OnSubscribeCache.java | 76 +++ .../rx/internal/util/CachedObservable.java | 432 ------------------ .../rx/internal/util/LinkedArrayList.java | 136 ------ .../operators/OnSubscribeCacheTest.java | 164 +++++++ .../internal/util/CachedObservableTest.java | 264 ----------- .../rx/internal/util/LinkedArrayListTest.java | 37 -- 7 files changed, 242 insertions(+), 871 deletions(-) create mode 100644 src/main/java/rx/internal/operators/OnSubscribeCache.java delete mode 100644 src/main/java/rx/internal/util/CachedObservable.java delete mode 100644 src/main/java/rx/internal/util/LinkedArrayList.java create mode 100644 src/test/java/rx/internal/operators/OnSubscribeCacheTest.java delete mode 100644 src/test/java/rx/internal/util/CachedObservableTest.java delete mode 100644 src/test/java/rx/internal/util/LinkedArrayListTest.java diff --git a/src/main/java/rx/Observable.java b/src/main/java/rx/Observable.java index 60d814186f..108246172a 100644 --- a/src/main/java/rx/Observable.java +++ b/src/main/java/rx/Observable.java @@ -3594,7 +3594,7 @@ public final Observable> buffer(Observable boundary, int initialC * @see ReactiveX operators documentation: Replay */ public final Observable cache() { - return CachedObservable.from(this); + return create(new OnSubscribeCache(this)); } /** @@ -3629,7 +3629,7 @@ public final Observable cache() { * @see ReactiveX operators documentation: Replay */ public final Observable cache(int capacityHint) { - return CachedObservable.from(this, capacityHint); + return create(new OnSubscribeCache(this, capacityHint)); } /** diff --git a/src/main/java/rx/internal/operators/OnSubscribeCache.java b/src/main/java/rx/internal/operators/OnSubscribeCache.java new file mode 100644 index 0000000000..a568fd0e0b --- /dev/null +++ b/src/main/java/rx/internal/operators/OnSubscribeCache.java @@ -0,0 +1,76 @@ +/** + * Copyright 2014 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.internal.operators; + +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; + +import rx.Observable; +import rx.Observable.OnSubscribe; +import rx.Subscriber; +import rx.subjects.ReplaySubject; +import rx.subjects.Subject; + +/** + * This method has similar behavior to {@link Observable#replay()} except that this auto-subscribes + * to the source Observable rather than returning a connectable Observable. + *

+ * + *

+ * This is useful with an Observable that you want to cache responses when you can't control the + * subscribe/unsubscribe behavior of all the Observers. + *

+ * Note: You sacrifice the ability to unsubscribe from the origin when you use this operator, so be + * careful not to use this operator on Observables that emit infinite or very large numbers of + * items, as this will use up memory. + * + * @param + * the cached value type + */ +public final class OnSubscribeCache implements OnSubscribe { + protected final Observable source; + protected final Subject cache; + volatile int sourceSubscribed; + @SuppressWarnings("rawtypes") + static final AtomicIntegerFieldUpdater SRC_SUBSCRIBED_UPDATER + = AtomicIntegerFieldUpdater.newUpdater(OnSubscribeCache.class, "sourceSubscribed"); + + public OnSubscribeCache(Observable source) { + this(source, ReplaySubject. create()); + } + + public OnSubscribeCache(Observable source, int capacity) { + this(source, ReplaySubject. create(capacity)); + } + + /* accessible to tests */OnSubscribeCache(Observable source, Subject cache) { + this.source = source; + this.cache = cache; + } + + @Override + public void call(Subscriber s) { + if (SRC_SUBSCRIBED_UPDATER.compareAndSet(this, 0, 1)) { + source.subscribe(cache); + /* + * Note that we will never unsubscribe from 'source' unless we receive `onCompleted` or `onError`, + * as we want to receive and cache all of its values. + * + * This means this should never be used on an infinite or very large sequence, similar to toList(). + */ + } + cache.unsafeSubscribe(s); + } +} diff --git a/src/main/java/rx/internal/util/CachedObservable.java b/src/main/java/rx/internal/util/CachedObservable.java deleted file mode 100644 index cda4b9d277..0000000000 --- a/src/main/java/rx/internal/util/CachedObservable.java +++ /dev/null @@ -1,432 +0,0 @@ -/** - * Copyright 2014 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.internal.util; - -import java.util.concurrent.atomic.*; - -import rx.*; -import rx.internal.operators.NotificationLite; -import rx.subscriptions.SerialSubscription; - -/** - * An observable which auto-connects to another observable, caches the elements - * from that observable but allows terminating the connection and completing the cache. - * - * @param the source element type - */ -public final class CachedObservable extends Observable { - /** The cache and replay state. */ - private CacheState state; - - /** - * Creates a cached Observable with a default capacity hint of 16. - * @param source the source Observable to cache - * @return the CachedObservable instance - */ - public static CachedObservable from(Observable source) { - return from(source, 16); - } - - /** - * Creates a cached Observable with the given capacity hint. - * @param source the source Observable to cache - * @param capacityHint the hint for the internal buffer size - * @return the CachedObservable instance - */ - public static CachedObservable from(Observable source, int capacityHint) { - if (capacityHint < 1) { - throw new IllegalArgumentException("capacityHint > 0 required"); - } - CacheState state = new CacheState(source, capacityHint); - CachedSubscribe onSubscribe = new CachedSubscribe(state); - return new CachedObservable(onSubscribe, state); - } - - /** - * Private constructor because state needs to be shared between the Observable body and - * the onSubscribe function. - * @param onSubscribe - * @param state - */ - private CachedObservable(OnSubscribe onSubscribe, CacheState state) { - super(onSubscribe); - this.state = state; - } - - /** - * Check if this cached observable is connected to its source. - * @return true if already connected - */ - /* public */boolean isConnected() { - return state.isConnected; - } - - /** - * Returns true if there are observers subscribed to this observable. - * @return - */ - /* public */ boolean hasObservers() { - return state.producers.length != 0; - } - - /** - * Returns the number of events currently cached. - * @return - */ - /* public */ int cachedEventCount() { - return state.size(); - } - - /** - * Contains the active child producers and the values to replay. - * - * @param - */ - static final class CacheState extends LinkedArrayList implements Observer { - /** The source observable to connect to. */ - final Observable source; - /** Holds onto the subscriber connected to source. */ - final SerialSubscription connection; - /** Guarded by connection (not this). */ - volatile ReplayProducer[] producers; - /** The default empty array of producers. */ - static final ReplayProducer[] EMPTY = new ReplayProducer[0]; - - final NotificationLite nl; - - /** Set to true after connection. */ - volatile boolean isConnected; - /** - * Indicates that the source has completed emitting values or the - * Observable was forcefully terminated. - */ - boolean sourceDone; - - public CacheState(Observable source, int capacityHint) { - super(capacityHint); - this.source = source; - this.producers = EMPTY; - this.nl = NotificationLite.instance(); - this.connection = new SerialSubscription(); - } - /** - * Adds a ReplayProducer to the producers array atomically. - * @param p - */ - public void addProducer(ReplayProducer p) { - // guarding by connection to save on allocating another object - // thus there are two distinct locks guarding the value-addition and child come-and-go - synchronized (connection) { - ReplayProducer[] a = producers; - int n = a.length; - ReplayProducer[] b = new ReplayProducer[n + 1]; - System.arraycopy(a, 0, b, 0, n); - b[n] = p; - producers = b; - } - } - /** - * Removes the ReplayProducer (if present) from the producers array atomically. - * @param p - */ - public void removeProducer(ReplayProducer p) { - synchronized (connection) { - ReplayProducer[] a = producers; - int n = a.length; - int j = -1; - for (int i = 0; i < n; i++) { - if (a[i].equals(p)) { - j = i; - break; - } - } - if (j < 0) { - return; - } - if (n == 1) { - producers = EMPTY; - return; - } - ReplayProducer[] b = new ReplayProducer[n - 1]; - System.arraycopy(a, 0, b, 0, j); - System.arraycopy(a, j + 1, b, j, n - j - 1); - producers = b; - } - } - /** - * Connects the cache to the source. - * Make sure this is called only once. - */ - public void connect() { - connection.set(source.subscribe(this)); - isConnected = true; - } - @Override - public void onNext(T t) { - Object o = nl.next(t); - synchronized (this) { - if (!sourceDone) { - add(o); - } else { - return; - } - } - dispatch(); - } - @Override - public void onError(Throwable e) { - Object o = nl.error(e); - synchronized (this) { - if (!sourceDone) { - sourceDone = true; - add(o); - } else { - return; - } - } - connection.unsubscribe(); - dispatch(); - } - @Override - public void onCompleted() { - Object o = nl.completed(); - synchronized (this) { - if (!sourceDone) { - sourceDone = true; - add(o); - } else { - return; - } - } - connection.unsubscribe(); - dispatch(); - } - /** - * Signals all known children there is work to do. - */ - void dispatch() { - ReplayProducer[] a = producers; - for (ReplayProducer rp : a) { - rp.replay(); - } - } - } - - /** - * Manages the subscription of child subscribers by setting up a replay producer and - * performs auto-connection of the very first subscription. - * @param the value type emitted - */ - static final class CachedSubscribe extends AtomicBoolean implements OnSubscribe { - /** */ - private static final long serialVersionUID = -2817751667698696782L; - final CacheState state; - public CachedSubscribe(CacheState state) { - this.state = state; - } - @Override - public void call(Subscriber t) { - // we can connect first because we replay everything anyway - ReplayProducer rp = new ReplayProducer(t, state); - state.addProducer(rp); - - t.add(rp); - t.setProducer(rp); - - // we ensure a single connection here to save an instance field of AtomicBoolean in state. - if (!get() && compareAndSet(false, true)) { - state.connect(); - } - - // no need to call rp.replay() here because the very first request will trigger it anyway - } - } - - /** - * Keeps track of the current request amount and the replay position for a child Subscriber. - * - * @param - */ - static final class ReplayProducer extends AtomicLong implements Producer, Subscription { - /** */ - private static final long serialVersionUID = -2557562030197141021L; - /** The actual child subscriber. */ - final Subscriber child; - /** The cache state object. */ - final CacheState state; - - /** - * Contains the reference to the buffer segment in replay. - * Accessed after reading state.size() and when emitting == true. - */ - Object[] currentBuffer; - /** - * Contains the index into the currentBuffer where the next value is expected. - * Accessed after reading state.size() and when emitting == true. - */ - int currentIndexInBuffer; - /** - * Contains the absolute index up until the values have been replayed so far. - */ - int index; - - /** Indicates there is a replay going on; guarded by this. */ - boolean emitting; - /** Indicates there were some state changes/replay attempts; guarded by this. */ - boolean missed; - - public ReplayProducer(Subscriber child, CacheState state) { - this.child = child; - this.state = state; - } - @Override - public void request(long n) { - for (;;) { - long r = get(); - if (r < 0) { - return; - } - long u = r + n; - if (u < 0) { - u = Long.MAX_VALUE; - } - if (compareAndSet(r, u)) { - replay(); - return; - } - } - } - /** - * Updates the request count to reflect values have been produced. - * @param n - * @return - */ - public long produced(long n) { - return addAndGet(-n); - } - - @Override - public boolean isUnsubscribed() { - return get() < 0; - } - @Override - public void unsubscribe() { - long r = get(); - if (r >= 0) { - r = getAndSet(-1L); // unsubscribed state is negative - if (r >= 0) { - state.removeProducer(this); - } - } - } - - /** - * Continue replaying available values if there are requests for them. - */ - public void replay() { - // make sure there is only a single thread emitting - synchronized (this) { - if (emitting) { - missed = true; - return; - } - emitting = true; - } - boolean skipFinal = false; - try { - final NotificationLite nl = state.nl; - final Subscriber child = this.child; - - for (;;) { - - long r = get(); - // read the size, if it is non-zero, we can safely read the head and - // read values up to the given absolute index - int s = state.size(); - if (s != 0) { - Object[] b = currentBuffer; - - // latch onto the very first buffer now that it is available. - if (b == null) { - b = state.head(); - currentBuffer = b; - } - final int n = b.length - 1; - int j = index; - int k = currentIndexInBuffer; - // eagerly emit any terminal event - if (r == 0) { - Object o = b[k]; - if (nl.isCompleted(o)) { - child.onCompleted(); - skipFinal = true; - unsubscribe(); - return; - } else - if (nl.isError(o)) { - child.onError(nl.getError(o)); - skipFinal = true; - unsubscribe(); - return; - } - } else - if (r > 0) { - int valuesProduced = 0; - - while (j < s && r > 0 && !child.isUnsubscribed()) { - if (k == n) { - b = (Object[])b[n]; - k = 0; - } - Object o = b[k]; - - if (nl.accept(child, o)) { - skipFinal = true; - unsubscribe(); - return; - } - - k++; - j++; - r--; - valuesProduced++; - } - - index = j; - currentIndexInBuffer = k; - currentBuffer = b; - produced(valuesProduced); - } - } - - synchronized (this) { - if (!missed) { - emitting = false; - skipFinal = true; - return; - } - missed = false; - } - } - } finally { - if (!skipFinal) { - synchronized (this) { - emitting = false; - } - } - } - } - } -} diff --git a/src/main/java/rx/internal/util/LinkedArrayList.java b/src/main/java/rx/internal/util/LinkedArrayList.java deleted file mode 100644 index 57a1289640..0000000000 --- a/src/main/java/rx/internal/util/LinkedArrayList.java +++ /dev/null @@ -1,136 +0,0 @@ -/** - * Copyright 2014 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.internal.util; - -import java.util.*; - -/** - * A list implementation which combines an ArrayList with a LinkedList to - * avoid copying values when the capacity needs to be increased. - *

- * The class is non final to allow embedding it directly and thus saving on object allocation. - */ -public class LinkedArrayList { - /** The capacity of each array segment. */ - final int capacityHint; - /** - * Contains the head of the linked array list if not null. The - * length is always capacityHint + 1 and the last element is an Object[] pointing - * to the next element of the linked array list. - */ - Object[] head; - /** The tail array where new elements will be added. */ - Object[] tail; - /** - * The total size of the list; written after elements have been added (release) and - * and when read, the value indicates how many elements can be safely read (acquire). - */ - volatile int size; - /** The next available slot in the current tail. */ - int indexInTail; - /** - * Constructor with the capacity hint of each array segment. - * @param capacityHint - */ - public LinkedArrayList(int capacityHint) { - this.capacityHint = capacityHint; - } - /** - * Adds a new element to this list. - * @param o the object to add, nulls are accepted - */ - public void add(Object o) { - // if no value yet, create the first array - if (size == 0) { - head = new Object[capacityHint + 1]; - tail = head; - head[0] = o; - indexInTail = 1; - size = 1; - } else - // if the tail is full, create a new tail and link - if (indexInTail == capacityHint) { - Object[] t = new Object[capacityHint + 1]; - t[0] = o; - tail[capacityHint] = t; - tail = t; - indexInTail = 1; - size++; - } else { - tail[indexInTail] = o; - indexInTail++; - size++; - } - } - /** - * Returns the head buffer segment or null if the list is empty. - * @return - */ - public Object[] head() { - return head; - } - /** - * Returns the tail buffer segment or null if the list is empty. - * @return - */ - public Object[] tail() { - return tail; - } - /** - * Returns the total size of the list. - * @return - */ - public int size() { - return size; - } - /** - * Returns the index of the next slot in the tail buffer segment. - * @return - */ - public int indexInTail() { - return indexInTail; - } - /** - * Returns the capacity hint that indicates the capacity of each buffer segment. - * @return - */ - public int capacityHint() { - return capacityHint; - } - /* Test support */List toList() { - final int cap = capacityHint; - final int s = size; - final List list = new ArrayList(s + 1); - - Object[] h = head(); - int j = 0; - int k = 0; - while (j < s) { - list.add(h[k]); - j++; - if (++k == cap) { - k = 0; - h = (Object[])h[cap]; - } - } - - return list; - } - @Override - public String toString() { - return toList().toString(); - } -} \ No newline at end of file diff --git a/src/test/java/rx/internal/operators/OnSubscribeCacheTest.java b/src/test/java/rx/internal/operators/OnSubscribeCacheTest.java new file mode 100644 index 0000000000..0d74cd878b --- /dev/null +++ b/src/test/java/rx/internal/operators/OnSubscribeCacheTest.java @@ -0,0 +1,164 @@ +/** + * Copyright 2014 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.internal.operators; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import java.util.Arrays; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import org.junit.Test; + +import rx.Observable; +import rx.Subscriber; +import rx.functions.Action0; +import rx.functions.Action1; +import rx.functions.Func1; +import rx.functions.Func2; +import rx.observers.TestSubscriber; +import rx.schedulers.Schedulers; +import rx.subjects.AsyncSubject; +import rx.subjects.BehaviorSubject; +import rx.subjects.PublishSubject; +import rx.subjects.ReplaySubject; +import rx.subjects.Subject; + +public class OnSubscribeCacheTest { + + @Test + public void testCache() throws InterruptedException { + final AtomicInteger counter = new AtomicInteger(); + Observable o = Observable.create(new Observable.OnSubscribe() { + + @Override + public void call(final Subscriber observer) { + new Thread(new Runnable() { + + @Override + public void run() { + counter.incrementAndGet(); + System.out.println("published observable being executed"); + observer.onNext("one"); + observer.onCompleted(); + } + }).start(); + } + }).cache(); + + // we then expect the following 2 subscriptions to get that same value + final CountDownLatch latch = new CountDownLatch(2); + + // subscribe once + o.subscribe(new Action1() { + + @Override + public void call(String v) { + assertEquals("one", v); + System.out.println("v: " + v); + latch.countDown(); + } + }); + + // subscribe again + o.subscribe(new Action1() { + + @Override + public void call(String v) { + assertEquals("one", v); + System.out.println("v: " + v); + latch.countDown(); + } + }); + + if (!latch.await(1000, TimeUnit.MILLISECONDS)) { + fail("subscriptions did not receive values"); + } + assertEquals(1, counter.get()); + } + + private void testWithCustomSubjectAndRepeat(Subject subject, Integer... expected) { + Observable source0 = Observable.just(1, 2, 3) + .subscribeOn(Schedulers.io()) + .flatMap(new Func1>() { + @Override + public Observable call(final Integer i) { + return Observable.timer(i * 20, TimeUnit.MILLISECONDS).map(new Func1() { + @Override + public Integer call(Long t1) { + return i; + } + }); + } + }); + + Observable source1 = Observable.create(new OnSubscribeCache(source0, subject)); + + Observable source2 = source1 + .repeat(4) + .zipWith(Observable.interval(0, 10, TimeUnit.MILLISECONDS, Schedulers.newThread()), new Func2() { + @Override + public Integer call(Integer t1, Long t2) { + return t1; + } + + }); + TestSubscriber ts = new TestSubscriber(); + source2.subscribe(ts); + + ts.awaitTerminalEvent(); + ts.assertNoErrors(); + System.out.println(ts.getOnNextEvents()); + ts.assertReceivedOnNext(Arrays.asList(expected)); + } + + @Test(timeout = 10000) + public void testWithAsyncSubjectAndRepeat() { + testWithCustomSubjectAndRepeat(AsyncSubject. create(), 3, 3, 3, 3); + } + + @Test(timeout = 10000) + public void testWithBehaviorSubjectAndRepeat() { + // BehaviorSubject just completes when repeated + testWithCustomSubjectAndRepeat(BehaviorSubject.create(0), 0, 1, 2, 3); + } + + @Test(timeout = 10000) + public void testWithPublishSubjectAndRepeat() { + // PublishSubject just completes when repeated + testWithCustomSubjectAndRepeat(PublishSubject. create(), 1, 2, 3); + } + + @Test + public void testWithReplaySubjectAndRepeat() { + testWithCustomSubjectAndRepeat(ReplaySubject. create(), 1, 2, 3, 1, 2, 3, 1, 2, 3, 1, 2, 3); + } + + @Test + public void testUnsubscribeSource() { + Action0 unsubscribe = mock(Action0.class); + Observable o = Observable.just(1).doOnUnsubscribe(unsubscribe).cache(); + o.subscribe(); + o.subscribe(); + o.subscribe(); + verify(unsubscribe, times(1)).call(); + } +} diff --git a/src/test/java/rx/internal/util/CachedObservableTest.java b/src/test/java/rx/internal/util/CachedObservableTest.java deleted file mode 100644 index c14018390f..0000000000 --- a/src/test/java/rx/internal/util/CachedObservableTest.java +++ /dev/null @@ -1,264 +0,0 @@ -/** - * Copyright 2014 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.internal.util; - -import static org.junit.Assert.*; -import static org.mockito.Mockito.*; - -import java.util.*; -import java.util.concurrent.*; -import java.util.concurrent.atomic.AtomicInteger; - -import org.junit.*; - -import rx.*; -import rx.Observable.OnSubscribe; -import rx.Observable; -import rx.exceptions.TestException; -import rx.functions.*; -import rx.observers.TestSubscriber; -import rx.schedulers.Schedulers; - -public class CachedObservableTest { - @Test - public void testColdReplayNoBackpressure() { - CachedObservable source = CachedObservable.from(Observable.range(0, 1000)); - - assertFalse("Source is connected!", source.isConnected()); - - TestSubscriber ts = new TestSubscriber(); - - source.subscribe(ts); - - assertTrue("Source is not connected!", source.isConnected()); - assertFalse("Subscribers retained!", source.hasObservers()); - - ts.assertNoErrors(); - ts.assertTerminalEvent(); - List onNextEvents = ts.getOnNextEvents(); - assertEquals(1000, onNextEvents.size()); - - for (int i = 0; i < 1000; i++) { - assertEquals((Integer)i, onNextEvents.get(i)); - } - } - @Test - public void testColdReplayBackpressure() { - CachedObservable source = CachedObservable.from(Observable.range(0, 1000)); - - assertFalse("Source is connected!", source.isConnected()); - - TestSubscriber ts = new TestSubscriber(); - ts.requestMore(10); - - source.subscribe(ts); - - assertTrue("Source is not connected!", source.isConnected()); - assertTrue("Subscribers not retained!", source.hasObservers()); - - ts.assertNoErrors(); - assertTrue(ts.getOnCompletedEvents().isEmpty()); - List onNextEvents = ts.getOnNextEvents(); - assertEquals(10, onNextEvents.size()); - - for (int i = 0; i < 10; i++) { - assertEquals((Integer)i, onNextEvents.get(i)); - } - - ts.unsubscribe(); - assertFalse("Subscribers retained!", source.hasObservers()); - } - - @Test - public void testCache() throws InterruptedException { - final AtomicInteger counter = new AtomicInteger(); - Observable o = Observable.create(new Observable.OnSubscribe() { - - @Override - public void call(final Subscriber observer) { - new Thread(new Runnable() { - - @Override - public void run() { - counter.incrementAndGet(); - System.out.println("published observable being executed"); - observer.onNext("one"); - observer.onCompleted(); - } - }).start(); - } - }).cache(); - - // we then expect the following 2 subscriptions to get that same value - final CountDownLatch latch = new CountDownLatch(2); - - // subscribe once - o.subscribe(new Action1() { - - @Override - public void call(String v) { - assertEquals("one", v); - System.out.println("v: " + v); - latch.countDown(); - } - }); - - // subscribe again - o.subscribe(new Action1() { - - @Override - public void call(String v) { - assertEquals("one", v); - System.out.println("v: " + v); - latch.countDown(); - } - }); - - if (!latch.await(1000, TimeUnit.MILLISECONDS)) { - fail("subscriptions did not receive values"); - } - assertEquals(1, counter.get()); - } - - @Test - public void testUnsubscribeSource() { - Action0 unsubscribe = mock(Action0.class); - Observable o = Observable.just(1).doOnUnsubscribe(unsubscribe).cache(); - o.subscribe(); - o.subscribe(); - o.subscribe(); - verify(unsubscribe, times(1)).call(); - } - - @Test - public void testTake() { - TestSubscriber ts = new TestSubscriber(); - - CachedObservable cached = CachedObservable.from(Observable.range(1, 100)); - cached.take(10).subscribe(ts); - - ts.assertNoErrors(); - ts.assertTerminalEvent(); - ts.assertReceivedOnNext(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)); - ts.assertUnsubscribed(); - assertFalse(cached.hasObservers()); - } - - @Test - public void testAsync() { - Observable source = Observable.range(1, 10000); - for (int i = 0; i < 100; i++) { - TestSubscriber ts1 = new TestSubscriber(); - - CachedObservable cached = CachedObservable.from(source); - - cached.observeOn(Schedulers.computation()).subscribe(ts1); - - ts1.awaitTerminalEvent(2, TimeUnit.SECONDS); - ts1.assertNoErrors(); - ts1.assertTerminalEvent(); - assertEquals(10000, ts1.getOnNextEvents().size()); - - TestSubscriber ts2 = new TestSubscriber(); - cached.observeOn(Schedulers.computation()).subscribe(ts2); - - ts2.awaitTerminalEvent(2, TimeUnit.SECONDS); - ts2.assertNoErrors(); - ts2.assertTerminalEvent(); - assertEquals(10000, ts2.getOnNextEvents().size()); - } - } - @Test - public void testAsyncComeAndGo() { - Observable source = Observable.timer(1, 1, TimeUnit.MILLISECONDS) - .take(1000) - .subscribeOn(Schedulers.io()); - CachedObservable cached = CachedObservable.from(source); - - Observable output = cached.observeOn(Schedulers.computation()); - - List> list = new ArrayList>(100); - for (int i = 0; i < 100; i++) { - TestSubscriber ts = new TestSubscriber(); - list.add(ts); - output.skip(i * 10).take(10).subscribe(ts); - } - - List expected = new ArrayList(); - for (int i = 0; i < 10; i++) { - expected.add((long)(i - 10)); - } - int j = 0; - for (TestSubscriber ts : list) { - ts.awaitTerminalEvent(3, TimeUnit.SECONDS); - ts.assertNoErrors(); - ts.assertTerminalEvent(); - - for (int i = j * 10; i < j * 10 + 10; i++) { - expected.set(i - j * 10, (long)i); - } - - ts.assertReceivedOnNext(expected); - - j++; - } - } - - @Test - public void testNoMissingBackpressureException() { - final int m = 4 * 1000 * 1000; - Observable firehose = Observable.create(new OnSubscribe() { - @Override - public void call(Subscriber t) { - for (int i = 0; i < m; i++) { - t.onNext(i); - } - t.onCompleted(); - } - }); - - TestSubscriber ts = new TestSubscriber(); - firehose.cache().observeOn(Schedulers.computation()).takeLast(100).subscribe(ts); - - ts.awaitTerminalEvent(3, TimeUnit.SECONDS); - ts.assertNoErrors(); - ts.assertTerminalEvent(); - - assertEquals(100, ts.getOnNextEvents().size()); - } - - @Test - public void testValuesAndThenError() { - Observable source = Observable.range(1, 10) - .concatWith(Observable.error(new TestException())) - .cache(); - - - TestSubscriber ts = new TestSubscriber(); - source.subscribe(ts); - - ts.assertReceivedOnNext(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)); - Assert.assertTrue(ts.getOnCompletedEvents().isEmpty()); - Assert.assertEquals(1, ts.getOnErrorEvents().size()); - - TestSubscriber ts2 = new TestSubscriber(); - source.subscribe(ts2); - - ts2.assertReceivedOnNext(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)); - Assert.assertTrue(ts2.getOnCompletedEvents().isEmpty()); - Assert.assertEquals(1, ts2.getOnErrorEvents().size()); - } -} diff --git a/src/test/java/rx/internal/util/LinkedArrayListTest.java b/src/test/java/rx/internal/util/LinkedArrayListTest.java deleted file mode 100644 index af7e167c19..0000000000 --- a/src/test/java/rx/internal/util/LinkedArrayListTest.java +++ /dev/null @@ -1,37 +0,0 @@ -/** - * Copyright 2014 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.internal.util; - -import java.util.*; -import static org.junit.Assert.*; - -import org.junit.Test; - -public class LinkedArrayListTest { - @Test - public void testAdd() { - LinkedArrayList list = new LinkedArrayList(16); - - List expected = new ArrayList(32); - for (int i = 0; i < 32; i++) { - list.add(i); - expected.add(i); - } - - assertEquals(expected, list.toList()); - assertEquals(32, list.size()); - } -} From ad29ab2d120afeae7c0f06a665fb91e46ddc1620 Mon Sep 17 00:00:00 2001 From: akarnokd Date: Mon, 20 Jul 2015 19:04:15 +0200 Subject: [PATCH 06/81] Fix autoConnect calling onStart twice. --- .../java/rx/internal/operators/OnSubscribeAutoConnect.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/main/java/rx/internal/operators/OnSubscribeAutoConnect.java b/src/main/java/rx/internal/operators/OnSubscribeAutoConnect.java index c664717332..75ea9c82cf 100644 --- a/src/main/java/rx/internal/operators/OnSubscribeAutoConnect.java +++ b/src/main/java/rx/internal/operators/OnSubscribeAutoConnect.java @@ -18,9 +18,11 @@ import java.util.concurrent.atomic.AtomicInteger; import rx.Observable.OnSubscribe; -import rx.*; +import rx.Subscriber; +import rx.Subscription; import rx.functions.Action1; import rx.observables.ConnectableObservable; +import rx.observers.Subscribers; /** * Wraps a ConnectableObservable and calls its connect() method once @@ -47,7 +49,7 @@ public OnSubscribeAutoConnect(ConnectableObservable source, } @Override public void call(Subscriber child) { - source.unsafeSubscribe(child); + source.unsafeSubscribe(Subscribers.wrap(child)); if (clients.incrementAndGet() == numberOfSubscribers) { source.connect(connection); } From aa6361a571b292e5273f9c9d28cfc29e0b65c89e Mon Sep 17 00:00:00 2001 From: Ben Christensen Date: Mon, 20 Jul 2015 10:03:30 -0700 Subject: [PATCH 07/81] Private toObservable renamed to asObservable - Making room for the public toObservable method. --- src/main/java/rx/Single.java | 56 ++++++++++++++++++------------------ 1 file changed, 28 insertions(+), 28 deletions(-) diff --git a/src/main/java/rx/Single.java b/src/main/java/rx/Single.java index 2aad23fb9a..c1dd59bcb8 100644 --- a/src/main/java/rx/Single.java +++ b/src/main/java/rx/Single.java @@ -244,7 +244,7 @@ public static interface Transformer extends Func1, Single> { * * @warn more complete description needed */ - private static Observable toObservable(Single t) { + private static Observable asObservable(Single t) { // is this sufficient, or do I need to keep the outer Single and subscribe to it? return Observable.create(t.onSubscribe); } @@ -265,7 +265,7 @@ private static Observable toObservable(Single t) { * @see ReactiveX operators documentation: To */ private final Single> nest() { - return Single.just(toObservable(this)); + return Single.just(asObservable(this)); } /* ********************************************************************************************************* @@ -290,7 +290,7 @@ private final Single> nest() { * @see ReactiveX operators documentation: Concat */ public final static Observable concat(Single t1, Single t2) { - return Observable.concat(toObservable(t1), toObservable(t2)); + return Observable.concat(asObservable(t1), asObservable(t2)); } /** @@ -312,7 +312,7 @@ public final static Observable concat(Single t1, SingleReactiveX operators documentation: Concat */ public final static Observable concat(Single t1, Single t2, Single t3) { - return Observable.concat(toObservable(t1), toObservable(t2), toObservable(t3)); + return Observable.concat(asObservable(t1), asObservable(t2), asObservable(t3)); } /** @@ -336,7 +336,7 @@ public final static Observable concat(Single t1, SingleReactiveX operators documentation: Concat */ public final static Observable concat(Single t1, Single t2, Single t3, Single t4) { - return Observable.concat(toObservable(t1), toObservable(t2), toObservable(t3), toObservable(t4)); + return Observable.concat(asObservable(t1), asObservable(t2), asObservable(t3), asObservable(t4)); } /** @@ -362,7 +362,7 @@ public final static Observable concat(Single t1, SingleReactiveX operators documentation: Concat */ public final static Observable concat(Single t1, Single t2, Single t3, Single t4, Single t5) { - return Observable.concat(toObservable(t1), toObservable(t2), toObservable(t3), toObservable(t4), toObservable(t5)); + return Observable.concat(asObservable(t1), asObservable(t2), asObservable(t3), asObservable(t4), asObservable(t5)); } /** @@ -390,7 +390,7 @@ public final static Observable concat(Single t1, SingleReactiveX operators documentation: Concat */ public final static Observable concat(Single t1, Single t2, Single t3, Single t4, Single t5, Single t6) { - return Observable.concat(toObservable(t1), toObservable(t2), toObservable(t3), toObservable(t4), toObservable(t5), toObservable(t6)); + return Observable.concat(asObservable(t1), asObservable(t2), asObservable(t3), asObservable(t4), asObservable(t5), asObservable(t6)); } /** @@ -420,7 +420,7 @@ public final static Observable concat(Single t1, SingleReactiveX operators documentation: Concat */ public final static Observable concat(Single t1, Single t2, Single t3, Single t4, Single t5, Single t6, Single t7) { - return Observable.concat(toObservable(t1), toObservable(t2), toObservable(t3), toObservable(t4), toObservable(t5), toObservable(t6), toObservable(t7)); + return Observable.concat(asObservable(t1), asObservable(t2), asObservable(t3), asObservable(t4), asObservable(t5), asObservable(t6), asObservable(t7)); } /** @@ -452,7 +452,7 @@ public final static Observable concat(Single t1, SingleReactiveX operators documentation: Concat */ public final static Observable concat(Single t1, Single t2, Single t3, Single t4, Single t5, Single t6, Single t7, Single t8) { - return Observable.concat(toObservable(t1), toObservable(t2), toObservable(t3), toObservable(t4), toObservable(t5), toObservable(t6), toObservable(t7), toObservable(t8)); + return Observable.concat(asObservable(t1), asObservable(t2), asObservable(t3), asObservable(t4), asObservable(t5), asObservable(t6), asObservable(t7), asObservable(t8)); } /** @@ -486,7 +486,7 @@ public final static Observable concat(Single t1, SingleReactiveX operators documentation: Concat */ public final static Observable concat(Single t1, Single t2, Single t3, Single t4, Single t5, Single t6, Single t7, Single t8, Single t9) { - return Observable.concat(toObservable(t1), toObservable(t2), toObservable(t3), toObservable(t4), toObservable(t5), toObservable(t6), toObservable(t7), toObservable(t8), toObservable(t9)); + return Observable.concat(asObservable(t1), asObservable(t2), asObservable(t3), asObservable(t4), asObservable(t5), asObservable(t6), asObservable(t7), asObservable(t8), asObservable(t9)); } /** @@ -694,7 +694,7 @@ public void onError(Throwable error) { * @see ReactiveX operators documentation: Merge */ public final static Observable merge(Single t1, Single t2) { - return Observable.merge(toObservable(t1), toObservable(t2)); + return Observable.merge(asObservable(t1), asObservable(t2)); } /** @@ -719,7 +719,7 @@ public final static Observable merge(Single t1, SingleReactiveX operators documentation: Merge */ public final static Observable merge(Single t1, Single t2, Single t3) { - return Observable.merge(toObservable(t1), toObservable(t2), toObservable(t3)); + return Observable.merge(asObservable(t1), asObservable(t2), asObservable(t3)); } /** @@ -746,7 +746,7 @@ public final static Observable merge(Single t1, SingleReactiveX operators documentation: Merge */ public final static Observable merge(Single t1, Single t2, Single t3, Single t4) { - return Observable.merge(toObservable(t1), toObservable(t2), toObservable(t3), toObservable(t4)); + return Observable.merge(asObservable(t1), asObservable(t2), asObservable(t3), asObservable(t4)); } /** @@ -775,7 +775,7 @@ public final static Observable merge(Single t1, SingleReactiveX operators documentation: Merge */ public final static Observable merge(Single t1, Single t2, Single t3, Single t4, Single t5) { - return Observable.merge(toObservable(t1), toObservable(t2), toObservable(t3), toObservable(t4), toObservable(t5)); + return Observable.merge(asObservable(t1), asObservable(t2), asObservable(t3), asObservable(t4), asObservable(t5)); } /** @@ -806,7 +806,7 @@ public final static Observable merge(Single t1, SingleReactiveX operators documentation: Merge */ public final static Observable merge(Single t1, Single t2, Single t3, Single t4, Single t5, Single t6) { - return Observable.merge(toObservable(t1), toObservable(t2), toObservable(t3), toObservable(t4), toObservable(t5), toObservable(t6)); + return Observable.merge(asObservable(t1), asObservable(t2), asObservable(t3), asObservable(t4), asObservable(t5), asObservable(t6)); } /** @@ -839,7 +839,7 @@ public final static Observable merge(Single t1, SingleReactiveX operators documentation: Merge */ public final static Observable merge(Single t1, Single t2, Single t3, Single t4, Single t5, Single t6, Single t7) { - return Observable.merge(toObservable(t1), toObservable(t2), toObservable(t3), toObservable(t4), toObservable(t5), toObservable(t6), toObservable(t7)); + return Observable.merge(asObservable(t1), asObservable(t2), asObservable(t3), asObservable(t4), asObservable(t5), asObservable(t6), asObservable(t7)); } /** @@ -874,7 +874,7 @@ public final static Observable merge(Single t1, SingleReactiveX operators documentation: Merge */ public final static Observable merge(Single t1, Single t2, Single t3, Single t4, Single t5, Single t6, Single t7, Single t8) { - return Observable.merge(toObservable(t1), toObservable(t2), toObservable(t3), toObservable(t4), toObservable(t5), toObservable(t6), toObservable(t7), toObservable(t8)); + return Observable.merge(asObservable(t1), asObservable(t2), asObservable(t3), asObservable(t4), asObservable(t5), asObservable(t6), asObservable(t7), asObservable(t8)); } /** @@ -911,7 +911,7 @@ public final static Observable merge(Single t1, SingleReactiveX operators documentation: Merge */ public final static Observable merge(Single t1, Single t2, Single t3, Single t4, Single t5, Single t6, Single t7, Single t8, Single t9) { - return Observable.merge(toObservable(t1), toObservable(t2), toObservable(t3), toObservable(t4), toObservable(t5), toObservable(t6), toObservable(t7), toObservable(t8), toObservable(t9)); + return Observable.merge(asObservable(t1), asObservable(t2), asObservable(t3), asObservable(t4), asObservable(t5), asObservable(t6), asObservable(t7), asObservable(t8), asObservable(t9)); } /** @@ -935,7 +935,7 @@ public final static Observable merge(Single t1, SingleReactiveX operators documentation: Zip */ public final static Single zip(Single o1, Single o2, final Func2 zipFunction) { - return just(new Observable[] { toObservable(o1), toObservable(o2) }).lift(new OperatorZip(zipFunction)); + return just(new Observable[] { asObservable(o1), asObservable(o2) }).lift(new OperatorZip(zipFunction)); } /** @@ -961,7 +961,7 @@ public final static Single zip(Single o1, SingleReactiveX operators documentation: Zip */ public final static Single zip(Single o1, Single o2, Single o3, Func3 zipFunction) { - return just(new Observable[] { toObservable(o1), toObservable(o2), toObservable(o3) }).lift(new OperatorZip(zipFunction)); + return just(new Observable[] { asObservable(o1), asObservable(o2), asObservable(o3) }).lift(new OperatorZip(zipFunction)); } /** @@ -989,7 +989,7 @@ public final static Single zip(Single o1, Singl * @see ReactiveX operators documentation: Zip */ public final static Single zip(Single o1, Single o2, Single o3, Single o4, Func4 zipFunction) { - return just(new Observable[] { toObservable(o1), toObservable(o2), toObservable(o3), toObservable(o4) }).lift(new OperatorZip(zipFunction)); + return just(new Observable[] { asObservable(o1), asObservable(o2), asObservable(o3), asObservable(o4) }).lift(new OperatorZip(zipFunction)); } /** @@ -1019,7 +1019,7 @@ public final static Single zip(Single o1, S * @see ReactiveX operators documentation: Zip */ public final static Single zip(Single o1, Single o2, Single o3, Single o4, Single o5, Func5 zipFunction) { - return just(new Observable[] { toObservable(o1), toObservable(o2), toObservable(o3), toObservable(o4), toObservable(o5) }).lift(new OperatorZip(zipFunction)); + return just(new Observable[] { asObservable(o1), asObservable(o2), asObservable(o3), asObservable(o4), asObservable(o5) }).lift(new OperatorZip(zipFunction)); } /** @@ -1052,7 +1052,7 @@ public final static Single zip(Single o */ public final static Single zip(Single o1, Single o2, Single o3, Single o4, Single o5, Single o6, Func6 zipFunction) { - return just(new Observable[] { toObservable(o1), toObservable(o2), toObservable(o3), toObservable(o4), toObservable(o5), toObservable(o6) }).lift(new OperatorZip(zipFunction)); + return just(new Observable[] { asObservable(o1), asObservable(o2), asObservable(o3), asObservable(o4), asObservable(o5), asObservable(o6) }).lift(new OperatorZip(zipFunction)); } /** @@ -1087,7 +1087,7 @@ public final static Single zip(Single Single zip(Single o1, Single o2, Single o3, Single o4, Single o5, Single o6, Single o7, Func7 zipFunction) { - return just(new Observable[] { toObservable(o1), toObservable(o2), toObservable(o3), toObservable(o4), toObservable(o5), toObservable(o6), toObservable(o7) }).lift(new OperatorZip(zipFunction)); + return just(new Observable[] { asObservable(o1), asObservable(o2), asObservable(o3), asObservable(o4), asObservable(o5), asObservable(o6), asObservable(o7) }).lift(new OperatorZip(zipFunction)); } /** @@ -1124,7 +1124,7 @@ public final static Single zip(Single Single zip(Single o1, Single o2, Single o3, Single o4, Single o5, Single o6, Single o7, Single o8, Func8 zipFunction) { - return just(new Observable[] { toObservable(o1), toObservable(o2), toObservable(o3), toObservable(o4), toObservable(o5), toObservable(o6), toObservable(o7), toObservable(o8) }).lift(new OperatorZip(zipFunction)); + return just(new Observable[] { asObservable(o1), asObservable(o2), asObservable(o3), asObservable(o4), asObservable(o5), asObservable(o6), asObservable(o7), asObservable(o8) }).lift(new OperatorZip(zipFunction)); } /** @@ -1163,7 +1163,7 @@ public final static Single zip(Single Single zip(Single o1, Single o2, Single o3, Single o4, Single o5, Single o6, Single o7, Single o8, Single o9, Func9 zipFunction) { - return just(new Observable[] { toObservable(o1), toObservable(o2), toObservable(o3), toObservable(o4), toObservable(o5), toObservable(o6), toObservable(o7), toObservable(o8), toObservable(o9) }).lift(new OperatorZip(zipFunction)); + return just(new Observable[] { asObservable(o1), asObservable(o2), asObservable(o3), asObservable(o4), asObservable(o5), asObservable(o6), asObservable(o7), asObservable(o8), asObservable(o9) }).lift(new OperatorZip(zipFunction)); } /** @@ -1222,7 +1222,7 @@ public final Single flatMap(final Func1ReactiveX operators documentation: FlatMap */ public final Observable flatMapObservable(Func1> func) { - return Observable.merge(toObservable(map(func))); + return Observable.merge(asObservable(map(func))); } /** @@ -1748,7 +1748,7 @@ public final Single timeout(long timeout, TimeUnit timeUnit, Single error(new TimeoutException()); } - return lift(new OperatorTimeout(timeout, timeUnit, toObservable(other), scheduler)); + return lift(new OperatorTimeout(timeout, timeUnit, asObservable(other), scheduler)); } /** From 048b435b8198a5d4439341ea1452bde077c3d92f Mon Sep 17 00:00:00 2001 From: Ben Christensen Date: Mon, 20 Jul 2015 10:08:54 -0700 Subject: [PATCH 08/81] Single.toObservable --- src/main/java/rx/Single.java | 11 +++++++++++ src/test/java/rx/SingleTest.java | 9 +++++++++ 2 files changed, 20 insertions(+) diff --git a/src/main/java/rx/Single.java b/src/main/java/rx/Single.java index c1dd59bcb8..ede27c8eb6 100644 --- a/src/main/java/rx/Single.java +++ b/src/main/java/rx/Single.java @@ -1649,6 +1649,17 @@ public void onNext(T t) { public final Single subscribeOn(Scheduler scheduler) { return nest().lift(new OperatorSubscribeOn(scheduler)); } + + /** + * Converts this Single into an {@link Observable}. + *

+ * + * + * @return an {@link Observable} that emits a single item T. + */ + public final Observable toObservable() { + return asObservable(this); + } /** * Returns a Single that mirrors the source Single but applies a timeout policy for its emitted item. If it diff --git a/src/test/java/rx/SingleTest.java b/src/test/java/rx/SingleTest.java index 778feffb3c..1efd1ae5a7 100644 --- a/src/test/java/rx/SingleTest.java +++ b/src/test/java/rx/SingleTest.java @@ -452,4 +452,13 @@ public void onStart() { ts.assertValue("hello"); } + + @Test + public void testToObservable() { + Observable a = Single.just("a").toObservable(); + TestSubscriber ts = TestSubscriber.create(); + a.subscribe(ts); + ts.assertValue("a"); + ts.assertCompleted(); + } } From 59b539b139f587bc7566b3c89a265a1516a4e7d0 Mon Sep 17 00:00:00 2001 From: Ben Christensen Date: Mon, 20 Jul 2015 10:58:51 -0700 Subject: [PATCH 09/81] 1.0.13 --- CHANGES.md | 127 +++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 127 insertions(+) diff --git a/CHANGES.md b/CHANGES.md index a92a299b5e..3f6776cd46 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,5 +1,132 @@ # RxJava Releases # +### Version 1.0.13 – July 20th 2015 ([Maven Central](http://search.maven.org/#artifactdetails%7Cio.reactivex%7Crxjava%7C1.0.13%7C)) ### + +This release has quite a few bug fixes and some new functionality. Items of note are detailed here with the list of changes at the bottom. + +##### merge + +The `merge` operator went through a major rewrite to fix some edge case bugs in the previous version. This has been sitting for months going through review and performance testing due to the importance and ubiquity of its usage. It is believed this rewrite is now production ready and achieves the goal of being more correct (no known edge cases at this time) while retaining comparable performance and memory usage. + +Special thanks to @akarnokd for this as `merge` is a challenging one to implement. + +##### window fix and behavior change + +Unsubscription bugs were fixed in `window`. Along the way it also resulted in a fix to one of the `window` overloads that had a functional discrepancy. + +```java +window(Func0> closingSelector) +``` + +This is a small behavior change that corrects it. If you use this overload, please review the change to ensure your application is not affected by an assumption of the previously buggy behavior: https://github.com/ReactiveX/RxJava/pull/3039 + +Note that this behavior change only affects that particular overload while the broader bug fixes affect all `window` overloads. + +##### rx.Single + +After [much discussion](https://github.com/ReactiveX/RxJava/issues/1594) it was decided to add a new type to represent an `Observable` that emits a single item. Much bike-shedding led to the name `Single`. This was chosen because `Future`, `Promise` and `Task` are overused and already have nuanced connotations that differ from `rx.Single`, and we didn't want long, obnoxious names with `Observable` as a prefix or suffix. Read the issue thread if you want to dig into the long debates. + +If you want to understand the reasoning behind adding this type, you can read about it [in this comment](https://github.com/ReactiveX/RxJava/issues/1594#issuecomment-101300655). + +In short, request/response semantics are so common that it was decided worth creating a type that composes well with an `Observable` but only exposes request/response. The difference in behavior and comparability was also deemed worth having an alternative to `Future`. In particular, a `Single` is lazy whereas `Future` is eager. Additionally, merging of `Single`s becomes an `Observable`, whereas combining `Future`s always emits another `Future`. + +Note that the API is added in an `@Experimental` state. We are fairly confident this will stick around, but are holding final judgement until it is used more broadly. We will promote to a stable API in v1.1 or v1.2. + +Examples below demonstrate use of `Single`. + +```java +// Hello World +Single hello = Single.just("Hello World!"); +hello.subscribe(System.out::println); + +// Async request/response +Single one = getData(1); +Single two = getOtherData(2); + +// merge request/responses into an Observable of multiple values (not possible with Futures) +Observable merged = one.mergeWith(two); + +// zip request/responses into another Single (similar to combining 2 Futures) +Single zipped = one.zipWith(two, (a, b) -> a + b); + +// flatMap to a Single +Single flatMapSingle = one.flatMap(v -> { + return getOtherData(5); +}); + +// flatMap to an Observable +Observable flatMapObservable = one.flatMapObservable(v -> { + return Observable.just(1, 2, 3); +}); + +// toObservable +Observable toObservable = one.toObservable(); + +// toSingle +Single toSingle = Observable.just(1).toSingle(); + +public static Single getData(int id) { + return Single. create(s -> { + // do blocking IO + s.onSuccess("data_" + id); + }).subscribeOn(Schedulers.io()); +} + +public static Single getOtherData(int id) { + return Single. create(s -> { + // simulate non-blocking IO + new Thread(() -> { + try { + s.onSuccess("other_" + id); + } catch (Exception e) { + s.onError(e); + } + }).start(); + }); +} +``` + +##### ConnectableObservable.autoConnect + +A new feature was added to `ConnectableObservable` similar in behavior to `refCount()`, except that it doesn't disconnect when subscribers are lost. This is useful in triggering an "auto connect" once a certain number of subscribers have subscribed. + +The [JavaDocs](https://github.com/ReactiveX/RxJava/blob/1877fa7bbc176029bcb5af00d8a7715dfbb6d373/src/main/java/rx/observables/ConnectableObservable.java#L96) and [unit tests](https://github.com/ReactiveX/RxJava/blob/1.x/src/test/java/rx/observables/ConnectableObservableTest.java) are good places to understand the feature. + +##### Deprecated onBackpressureBlock + +The `onBackpressureBlock` operator has been deprecated. It will not ever be removed during the 1.x lifecycle, but it is recommended to not use it. It has proven to be a common source of deadlocks and is difficult to debug. It is instead recommended to use non-blocking approaches to backpressure, rather than callstack blocking. Approaches to backpressure and flow control are [discussed on the wiki](https://github.com/ReactiveX/RxJava/wiki/Backpressure). + +#### Changes + +* [Pull 3012] (https://github.com/ReactiveX/RxJava/pull/3012) rx.Single +* [Pull 2983] (https://github.com/ReactiveX/RxJava/pull/2983) Fixed multiple calls to onStart. +* [Pull 2970] (https://github.com/ReactiveX/RxJava/pull/2970) Deprecated onBackpressureBlock +* [Pull 2997] (https://github.com/ReactiveX/RxJava/pull/2997) Fix retry() race conditions +* [Pull 3028] (https://github.com/ReactiveX/RxJava/pull/3028) Delay: error cut ahead was not properly serialized +* [Pull 3042] (https://github.com/ReactiveX/RxJava/pull/3042) add backpressure support for defaultIfEmpty() +* [Pull 3049] (https://github.com/ReactiveX/RxJava/pull/3049) single: add toSingle method to Observable +* [Pull 3055] (https://github.com/ReactiveX/RxJava/pull/3055) toSingle() should use unsafeSubscribe +* [Pull 3023] (https://github.com/ReactiveX/RxJava/pull/3023) ConnectableObservable autoConnect operator +* [Pull 2928] (https://github.com/ReactiveX/RxJava/pull/2928) Merge and MergeMaxConcurrent unified and rewritten +* [Pull 3039] (https://github.com/ReactiveX/RxJava/pull/3039) Window with Observable: fixed unsubscription and behavior +* [Pull 3045] (https://github.com/ReactiveX/RxJava/pull/3045) ElementAt request management enhanced +* [Pull 3048] (https://github.com/ReactiveX/RxJava/pull/3048) CompositeException extra NPE protection +* [Pull 3052] (https://github.com/ReactiveX/RxJava/pull/3052) Reduce test failure likelihood of testMultiThreadedWithNPEinMiddle +* [Pull 3031] (https://github.com/ReactiveX/RxJava/pull/3031) Fix OperatorFlatMapPerf.flatMapIntPassthruAsync Perf Test +* [Pull 2975] (https://github.com/ReactiveX/RxJava/pull/2975) Deprecate and rename two timer overloads to interval +* [Pull 2982] (https://github.com/ReactiveX/RxJava/pull/2982) TestSubscriber - add factory methods +* [Pull 2995] (https://github.com/ReactiveX/RxJava/pull/2995) switchOnNext - ensure initial requests additive and fix request overflow +* [Pull 2972] (https://github.com/ReactiveX/RxJava/pull/2972) Fixed window(time) to work properly with unsubscription, added +* [Pull 2990] (https://github.com/ReactiveX/RxJava/pull/2990) Improve Subscriber readability +* [Pull 3018] (https://github.com/ReactiveX/RxJava/pull/3018) TestSubscriber - fix awaitTerminalEventAndUnsubscribeOnTimeout +* [Pull 3034] (https://github.com/ReactiveX/RxJava/pull/3034) Instantiate EMPTY lazily +* [Pull 3033] (https://github.com/ReactiveX/RxJava/pull/3033) takeLast() javadoc fixes, standardize parameter names (count instead of num) +* [Pull 3043] (https://github.com/ReactiveX/RxJava/pull/3043) TestSubscriber javadoc cleanup +* [Pull 3065] (https://github.com/ReactiveX/RxJava/pull/3065) add Subscribers.wrap +* [Pull 3091] (https://github.com/ReactiveX/RxJava/pull/3091) Fix autoConnect calling onStart twice. +* [Pull 3092] (https://github.com/ReactiveX/RxJava/pull/3092) Single.toObservable + + ### Version 1.0.12 – June 9th 2015 ([Maven Central](http://search.maven.org/#artifactdetails%7Cio.reactivex%7Crxjava%7C1.0.12%7C)) ### * [Pull 2963] (https://github.com/ReactiveX/RxJava/pull/2963) Set of standard producers and updated queue implementations From 79c7cd2604c481a5c05d5c393a6ab50384a9821a Mon Sep 17 00:00:00 2001 From: akarnokd Date: Mon, 20 Jul 2015 23:04:52 +0200 Subject: [PATCH 10/81] Fix request != 0 checking in the scalar paths of merge() --- .../rx/internal/operators/OperatorMerge.java | 6 +++-- src/test/java/rx/BackpressureTests.java | 24 +++++++++++++++++++ 2 files changed, 28 insertions(+), 2 deletions(-) diff --git a/src/main/java/rx/internal/operators/OperatorMerge.java b/src/main/java/rx/internal/operators/OperatorMerge.java index 98cb548391..d2f52cb204 100644 --- a/src/main/java/rx/internal/operators/OperatorMerge.java +++ b/src/main/java/rx/internal/operators/OperatorMerge.java @@ -315,7 +315,8 @@ void tryEmit(InnerSubscriber subscriber, T value) { if (r != 0L) { synchronized (this) { // if nobody is emitting and child has available requests - if (!emitting) { + r = producer.get(); + if (!emitting && r != 0L) { emitting = true; success = true; } @@ -422,7 +423,8 @@ void tryEmit(T value) { if (r != 0L) { synchronized (this) { // if nobody is emitting and child has available requests - if (!emitting) { + r = producer.get(); + if (!emitting && r != 0L) { emitting = true; success = true; } diff --git a/src/test/java/rx/BackpressureTests.java b/src/test/java/rx/BackpressureTests.java index ffa2e01129..439b18a08f 100644 --- a/src/test/java/rx/BackpressureTests.java +++ b/src/test/java/rx/BackpressureTests.java @@ -123,6 +123,30 @@ public void testMergeAsync() { assertTrue(c2.get() < RxRingBuffer.SIZE * 5); } + @Test + public void testMergeAsyncThenObserveOnLoop() { + for (int i = 0; i < 500; i++) { + if (i % 10 == 0) { + System.out.println("testMergeAsyncThenObserveOnLoop >> " + i); + } + // Verify there is no MissingBackpressureException + int NUM = (int) (RxRingBuffer.SIZE * 4.1); + AtomicInteger c1 = new AtomicInteger(); + AtomicInteger c2 = new AtomicInteger(); + + TestSubscriber ts = new TestSubscriber(); + Observable merged = Observable.merge( + incrementingIntegers(c1).subscribeOn(Schedulers.computation()), + incrementingIntegers(c2).subscribeOn(Schedulers.computation())); + + merged.observeOn(Schedulers.io()).take(NUM).subscribe(ts); + ts.awaitTerminalEvent(); + ts.assertNoErrors(); + System.out.println("testMergeAsyncThenObserveOn => Received: " + ts.getOnNextEvents().size() + " Emitted: " + c1.get() + " / " + c2.get()); + assertEquals(NUM, ts.getOnNextEvents().size()); + } + } + @Test public void testMergeAsyncThenObserveOn() { int NUM = (int) (RxRingBuffer.SIZE * 4.1); From ccddec4eea7d4fee9c81578351e9c35de478f618 Mon Sep 17 00:00:00 2001 From: Dave Moten Date: Fri, 17 Jul 2015 09:49:46 +1000 Subject: [PATCH 11/81] reduce probability of ExecutorSchedulerTest.testOnBackpressureDrop failing on slow machine --- src/test/java/rx/schedulers/ExecutorSchedulerTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/test/java/rx/schedulers/ExecutorSchedulerTest.java b/src/test/java/rx/schedulers/ExecutorSchedulerTest.java index b11f7879e1..cdefabc757 100644 --- a/src/test/java/rx/schedulers/ExecutorSchedulerTest.java +++ b/src/test/java/rx/schedulers/ExecutorSchedulerTest.java @@ -177,11 +177,11 @@ public void execute(Runnable command) { }; ExecutorSchedulerWorker w = (ExecutorSchedulerWorker)Schedulers.from(e).createWorker(); - w.schedule(Actions.empty(), 1, TimeUnit.MILLISECONDS); + w.schedule(Actions.empty(), 50, TimeUnit.MILLISECONDS); assertTrue(w.tasks.hasSubscriptions()); - Thread.sleep(100); + Thread.sleep(150); assertFalse(w.tasks.hasSubscriptions()); } From efefdb7e76758b2b310069d1f3a6ce8c2a5c297d Mon Sep 17 00:00:00 2001 From: Dave Moten Date: Fri, 17 Jul 2015 09:36:48 +1000 Subject: [PATCH 12/81] break tests as approach timeout so that don't fail on slow machines --- src/test/java/rx/BackpressureTests.java | 6 ++++++ .../internal/operators/OperatorMergeMaxConcurrentTest.java | 4 ++++ 2 files changed, 10 insertions(+) diff --git a/src/test/java/rx/BackpressureTests.java b/src/test/java/rx/BackpressureTests.java index 439b18a08f..e46dfebcb5 100644 --- a/src/test/java/rx/BackpressureTests.java +++ b/src/test/java/rx/BackpressureTests.java @@ -443,7 +443,13 @@ public void testFirehoseFailsAsExpected() { @Test(timeout = 10000) public void testOnBackpressureDrop() { + long t = System.currentTimeMillis(); for (int i = 0; i < 100; i++) { + // stop the test if we are getting close to the timeout because slow machines + // may not get through 100 iterations + if (System.currentTimeMillis() - t > TimeUnit.SECONDS.toMillis(9)) { + break; + } int NUM = (int) (RxRingBuffer.SIZE * 1.1); // > 1 so that take doesn't prevent buffer overflow AtomicInteger c = new AtomicInteger(); TestSubscriber ts = new TestSubscriber(); diff --git a/src/test/java/rx/internal/operators/OperatorMergeMaxConcurrentTest.java b/src/test/java/rx/internal/operators/OperatorMergeMaxConcurrentTest.java index 9cd65de8d0..af20d14316 100644 --- a/src/test/java/rx/internal/operators/OperatorMergeMaxConcurrentTest.java +++ b/src/test/java/rx/internal/operators/OperatorMergeMaxConcurrentTest.java @@ -224,7 +224,11 @@ public void testSimpleOneLessAsyncLoop() { } @Test(timeout = 10000) public void testSimpleOneLessAsync() { + long t = System.currentTimeMillis(); for (int i = 2; i < 50; i++) { + if (System.currentTimeMillis() - t > TimeUnit.SECONDS.toMillis(9)) { + break; + } TestSubscriber ts = new TestSubscriber(); List> sourceList = new ArrayList>(i); Set expected = new HashSet(i); From 336327f3d1e09de3d7767b99283a115de2725ce8 Mon Sep 17 00:00:00 2001 From: David Gross Date: Tue, 14 Jul 2015 13:23:54 -0700 Subject: [PATCH 13/81] Add "since" annotations to javadocs for new Experimental/Beta methods --- src/main/java/rx/Observable.java | 6 ++++++ src/main/java/rx/observers/Subscribers.java | 2 +- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/src/main/java/rx/Observable.java b/src/main/java/rx/Observable.java index 108246172a..ff4ffe8299 100644 --- a/src/main/java/rx/Observable.java +++ b/src/main/java/rx/Observable.java @@ -1750,6 +1750,7 @@ public final static Observable merge(ObservableReactiveX operators documentation: Merge + * @since (if this graduates from Experimental/Beta to supported, replace this parenthetical with the release number) */ @Experimental @SuppressWarnings({"unchecked", "rawtypes"}) @@ -2048,11 +2049,13 @@ public final static Observable merge(Observable[] sequences) * the maximum number of Observables that may be subscribed to concurrently * @return an Observable that emits all of the items emitted by the Observables in the Array * @see ReactiveX operators documentation: Merge + * @since (if this graduates from Experimental/Beta to supported, replace this parenthetical with the release number) */ @Experimental public final static Observable merge(Observable[] sequences, int maxConcurrent) { return merge(from(sequences), maxConcurrent); } + /** * Flattens an Observable that emits Observables into one Observable, in a way that allows an Observer to * receive all successfully emitted items from all of the source Observables without being interrupted by @@ -2080,6 +2083,7 @@ public final static Observable merge(Observable[] sequences, public final static Observable mergeDelayError(Observable> source) { return source.lift(OperatorMerge.instance(true)); } + /** * Flattens an Observable that emits Observables into one Observable, in a way that allows an Observer to * receive all successfully emitted items from all of the source Observables without being interrupted by @@ -2106,6 +2110,7 @@ public final static Observable mergeDelayError(ObservableReactiveX operators documentation: Merge + * @since (if this graduates from Experimental/Beta to supported, replace this parenthetical with the release number) */ @Experimental public final static Observable mergeDelayError(Observable> source, int maxConcurrent) { @@ -5519,6 +5524,7 @@ public final Observable onBackpressureDrop() { public final Observable onBackpressureBlock(int maxQueueLength) { return lift(new OperatorOnBackpressureBlock(maxQueueLength)); } + /** * Instructs an Observable that is emitting items faster than its observer can consume them to block the * producer thread if the number of undelivered onNext events reaches the system-wide ring buffer size. diff --git a/src/main/java/rx/observers/Subscribers.java b/src/main/java/rx/observers/Subscribers.java index 1c42aa4b68..4e81c1af8d 100644 --- a/src/main/java/rx/observers/Subscribers.java +++ b/src/main/java/rx/observers/Subscribers.java @@ -213,7 +213,7 @@ public final void onNext(T args) { * subscriber, has backpressure controlled by * subscriber and uses subscriber to * manage unsubscription. - * + * @since (if this graduates from Experimental/Beta to supported, replace this parenthetical with the release number) */ @Experimental public static Subscriber wrap(final Subscriber subscriber) { From b5cec41817781935c482b366b0eecf52cd15219d Mon Sep 17 00:00:00 2001 From: David Gross Date: Tue, 21 Jul 2015 12:43:36 -0700 Subject: [PATCH 14/81] window() behavior changed, so did marble diagram & thus its size --- src/main/java/rx/Observable.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/rx/Observable.java b/src/main/java/rx/Observable.java index ff4ffe8299..9fd476eab7 100644 --- a/src/main/java/rx/Observable.java +++ b/src/main/java/rx/Observable.java @@ -9155,7 +9155,7 @@ public final Observable withLatestFrom(Observable other, * Observable emits connected, non-overlapping windows. It emits the current window and opens a new one * whenever the Observable produced by the specified {@code closingSelector} emits an item. *

- * + * *

*
Backpressure Support:
*
This operator does not support backpressure as it uses the {@code closingSelector} to control data From 56da24dc3251b43970835e5c30e6066407bbde37 Mon Sep 17 00:00:00 2001 From: Dave Moten Date: Fri, 24 Jul 2015 18:05:49 +1000 Subject: [PATCH 15/81] remove OperatorOnErrorFlatMap because unused --- .../operators/OperatorOnErrorFlatMap.java | 84 ------------------- 1 file changed, 84 deletions(-) delete mode 100644 src/main/java/rx/internal/operators/OperatorOnErrorFlatMap.java diff --git a/src/main/java/rx/internal/operators/OperatorOnErrorFlatMap.java b/src/main/java/rx/internal/operators/OperatorOnErrorFlatMap.java deleted file mode 100644 index f8e56971f8..0000000000 --- a/src/main/java/rx/internal/operators/OperatorOnErrorFlatMap.java +++ /dev/null @@ -1,84 +0,0 @@ -/** - * Copyright 2014 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.internal.operators; - -import rx.Observable; -import rx.Observable.Operator; -import rx.Subscriber; -import rx.exceptions.OnErrorThrowable; -import rx.functions.Func1; -import rx.plugins.RxJavaPlugins; - -/** - * Allows inserting onNext events into a stream when onError events are received - * and continuing the original sequence instead of terminating. Thus it allows a sequence - * with multiple onError events. - */ -public final class OperatorOnErrorFlatMap implements Operator { - - private final Func1> resumeFunction; - - public OperatorOnErrorFlatMap(Func1> f) { - this.resumeFunction = f; - } - - @Override - public Subscriber call(final Subscriber child) { - return new Subscriber(child) { - - @Override - public void onCompleted() { - child.onCompleted(); - } - - @Override - public void onError(Throwable e) { - try { - RxJavaPlugins.getInstance().getErrorHandler().handleError(e); - Observable resume = resumeFunction.call(OnErrorThrowable.from(e)); - resume.unsafeSubscribe(new Subscriber() { - - @Override - public void onCompleted() { - // ignore as we will continue the parent Observable - } - - @Override - public void onError(Throwable e) { - // if the splice also fails we shut it all down - child.onError(e); - } - - @Override - public void onNext(T t) { - child.onNext(t); - } - - }); - } catch (Throwable e2) { - child.onError(e2); - } - } - - @Override - public void onNext(T t) { - child.onNext(t); - } - - }; - } - -} From e8860525f87572c6ae2cc1ac2000375b798c25e6 Mon Sep 17 00:00:00 2001 From: akarnokd Date: Fri, 24 Jul 2015 11:12:32 +0200 Subject: [PATCH 16/81] Unit tests and cleanup of JCTools' queues. --- .../util/atomic/MpscLinkedAtomicQueue.java | 2 +- .../util/atomic/SpscLinkedAtomicQueue.java | 2 +- .../internal/util/unsafe/MpmcArrayQueue.java | 20 +- .../internal/util/unsafe/MpscLinkedQueue.java | 2 +- .../internal/util/unsafe/SpmcArrayQueue.java | 20 +- .../internal/util/unsafe/SpscArrayQueue.java | 23 +- .../internal/util/unsafe/SpscLinkedQueue.java | 2 +- .../rx/internal/util/JCToolsQueueTests.java | 419 +++++++++++++++++- 8 files changed, 428 insertions(+), 62 deletions(-) diff --git a/src/main/java/rx/internal/util/atomic/MpscLinkedAtomicQueue.java b/src/main/java/rx/internal/util/atomic/MpscLinkedAtomicQueue.java index ebc1264599..261e4c2a1b 100644 --- a/src/main/java/rx/internal/util/atomic/MpscLinkedAtomicQueue.java +++ b/src/main/java/rx/internal/util/atomic/MpscLinkedAtomicQueue.java @@ -58,7 +58,7 @@ public MpscLinkedAtomicQueue() { @Override public final boolean offer(final E nextValue) { if (nextValue == null) { - throw new IllegalArgumentException("null elements not allowed"); + throw new NullPointerException("null elements not allowed"); } final LinkedQueueNode nextNode = new LinkedQueueNode(nextValue); final LinkedQueueNode prevProducerNode = xchgProducerNode(nextNode); diff --git a/src/main/java/rx/internal/util/atomic/SpscLinkedAtomicQueue.java b/src/main/java/rx/internal/util/atomic/SpscLinkedAtomicQueue.java index 5832f7371d..deb1ff7b68 100644 --- a/src/main/java/rx/internal/util/atomic/SpscLinkedAtomicQueue.java +++ b/src/main/java/rx/internal/util/atomic/SpscLinkedAtomicQueue.java @@ -59,7 +59,7 @@ public SpscLinkedAtomicQueue() { @Override public boolean offer(final E nextValue) { if (nextValue == null) { - throw new IllegalArgumentException("null elements not allowed"); + throw new NullPointerException("null elements not allowed"); } final LinkedQueueNode nextNode = new LinkedQueueNode(nextValue); lpProducerNode().soNext(nextNode); diff --git a/src/main/java/rx/internal/util/unsafe/MpmcArrayQueue.java b/src/main/java/rx/internal/util/unsafe/MpmcArrayQueue.java index 8333723036..a75c2a0028 100644 --- a/src/main/java/rx/internal/util/unsafe/MpmcArrayQueue.java +++ b/src/main/java/rx/internal/util/unsafe/MpmcArrayQueue.java @@ -28,15 +28,7 @@ public MpmcArrayQueueL1Pad(int capacity) { } abstract class MpmcArrayQueueProducerField extends MpmcArrayQueueL1Pad { - private final static long P_INDEX_OFFSET; - static { - try { - P_INDEX_OFFSET = UNSAFE.objectFieldOffset(MpmcArrayQueueProducerField.class - .getDeclaredField("producerIndex")); - } catch (NoSuchFieldException e) { - throw new RuntimeException(e); - } - } + private final static long P_INDEX_OFFSET = UnsafeAccess.addressOf(MpmcArrayQueueProducerField.class, "producerIndex"); private volatile long producerIndex; public MpmcArrayQueueProducerField(int capacity) { @@ -62,15 +54,7 @@ public MpmcArrayQueueL2Pad(int capacity) { } abstract class MpmcArrayQueueConsumerField extends MpmcArrayQueueL2Pad { - private final static long C_INDEX_OFFSET; - static { - try { - C_INDEX_OFFSET = UNSAFE.objectFieldOffset(MpmcArrayQueueConsumerField.class - .getDeclaredField("consumerIndex")); - } catch (NoSuchFieldException e) { - throw new RuntimeException(e); - } - } + private final static long C_INDEX_OFFSET = UnsafeAccess.addressOf(MpmcArrayQueueConsumerField.class, "consumerIndex"); private volatile long consumerIndex; public MpmcArrayQueueConsumerField(int capacity) { diff --git a/src/main/java/rx/internal/util/unsafe/MpscLinkedQueue.java b/src/main/java/rx/internal/util/unsafe/MpscLinkedQueue.java index f9e63f1c6b..2607c3a023 100644 --- a/src/main/java/rx/internal/util/unsafe/MpscLinkedQueue.java +++ b/src/main/java/rx/internal/util/unsafe/MpscLinkedQueue.java @@ -69,7 +69,7 @@ protected final LinkedQueueNode xchgProducerNode(LinkedQueueNode newVal) { @Override public final boolean offer(final E nextValue) { if (nextValue == null) { - throw new IllegalArgumentException("null elements not allowed"); + throw new NullPointerException("null elements not allowed"); } final LinkedQueueNode nextNode = new LinkedQueueNode(nextValue); final LinkedQueueNode prevProducerNode = xchgProducerNode(nextNode); diff --git a/src/main/java/rx/internal/util/unsafe/SpmcArrayQueue.java b/src/main/java/rx/internal/util/unsafe/SpmcArrayQueue.java index ebf79f0708..8a4251872d 100644 --- a/src/main/java/rx/internal/util/unsafe/SpmcArrayQueue.java +++ b/src/main/java/rx/internal/util/unsafe/SpmcArrayQueue.java @@ -28,15 +28,7 @@ public SpmcArrayQueueL1Pad(int capacity) { } abstract class SpmcArrayQueueProducerField extends SpmcArrayQueueL1Pad { - protected final static long P_INDEX_OFFSET; - static { - try { - P_INDEX_OFFSET = - UNSAFE.objectFieldOffset(SpmcArrayQueueProducerField.class.getDeclaredField("producerIndex")); - } catch (NoSuchFieldException e) { - throw new RuntimeException(e); - } - } + protected final static long P_INDEX_OFFSET = UnsafeAccess.addressOf(SpmcArrayQueueProducerField.class, "producerIndex"); private volatile long producerIndex; protected final long lvProducerIndex() { @@ -62,15 +54,7 @@ public SpmcArrayQueueL2Pad(int capacity) { } abstract class SpmcArrayQueueConsumerField extends SpmcArrayQueueL2Pad { - protected final static long C_INDEX_OFFSET; - static { - try { - C_INDEX_OFFSET = - UNSAFE.objectFieldOffset(SpmcArrayQueueConsumerField.class.getDeclaredField("consumerIndex")); - } catch (NoSuchFieldException e) { - throw new RuntimeException(e); - } - } + protected final static long C_INDEX_OFFSET = UnsafeAccess.addressOf(SpmcArrayQueueConsumerField.class, "consumerIndex"); private volatile long consumerIndex; public SpmcArrayQueueConsumerField(int capacity) { diff --git a/src/main/java/rx/internal/util/unsafe/SpscArrayQueue.java b/src/main/java/rx/internal/util/unsafe/SpscArrayQueue.java index 16d40b6951..88c6d491c6 100644 --- a/src/main/java/rx/internal/util/unsafe/SpscArrayQueue.java +++ b/src/main/java/rx/internal/util/unsafe/SpscArrayQueue.java @@ -36,15 +36,7 @@ public SpscArrayQueueL1Pad(int capacity) { } abstract class SpscArrayQueueProducerFields extends SpscArrayQueueL1Pad { - protected final static long P_INDEX_OFFSET; - static { - try { - P_INDEX_OFFSET = - UNSAFE.objectFieldOffset(SpscArrayQueueProducerFields.class.getDeclaredField("producerIndex")); - } catch (NoSuchFieldException e) { - throw new RuntimeException(e); - } - } + protected final static long P_INDEX_OFFSET = UnsafeAccess.addressOf(SpscArrayQueueProducerFields.class, "producerIndex"); protected long producerIndex; protected long producerLookAhead; @@ -64,15 +56,7 @@ public SpscArrayQueueL2Pad(int capacity) { abstract class SpscArrayQueueConsumerField extends SpscArrayQueueL2Pad { protected long consumerIndex; - protected final static long C_INDEX_OFFSET; - static { - try { - C_INDEX_OFFSET = - UNSAFE.objectFieldOffset(SpscArrayQueueConsumerField.class.getDeclaredField("consumerIndex")); - } catch (NoSuchFieldException e) { - throw new RuntimeException(e); - } - } + protected final static long C_INDEX_OFFSET = UnsafeAccess.addressOf(SpscArrayQueueConsumerField.class, "consumerIndex"); public SpscArrayQueueConsumerField(int capacity) { super(capacity); } @@ -116,6 +100,9 @@ public SpscArrayQueue(final int capacity) { */ @Override public boolean offer(final E e) { + if (e == null) { + throw new NullPointerException("null elements not allowed"); + } // local load of field to avoid repeated loads after volatile reads final E[] lElementBuffer = buffer; final long index = producerIndex; diff --git a/src/main/java/rx/internal/util/unsafe/SpscLinkedQueue.java b/src/main/java/rx/internal/util/unsafe/SpscLinkedQueue.java index 7c3c675b48..b9a037a986 100644 --- a/src/main/java/rx/internal/util/unsafe/SpscLinkedQueue.java +++ b/src/main/java/rx/internal/util/unsafe/SpscLinkedQueue.java @@ -62,7 +62,7 @@ public SpscLinkedQueue() { @Override public boolean offer(final E nextValue) { if (nextValue == null) { - throw new IllegalArgumentException("null elements not allowed"); + throw new NullPointerException("null elements not allowed"); } final LinkedQueueNode nextNode = new LinkedQueueNode(nextValue); producerNode.soNext(nextNode); diff --git a/src/test/java/rx/internal/util/JCToolsQueueTests.java b/src/test/java/rx/internal/util/JCToolsQueueTests.java index 2645dcd1c1..fea60217eb 100644 --- a/src/test/java/rx/internal/util/JCToolsQueueTests.java +++ b/src/test/java/rx/internal/util/JCToolsQueueTests.java @@ -17,13 +17,94 @@ import static org.junit.Assert.*; +import java.util.*; +import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicInteger; + import org.junit.Test; +import rx.internal.util.atomic.*; import rx.internal.util.unsafe.*; public class JCToolsQueueTests { + static final class IntField { + int value; + } + static void await(CyclicBarrier cb) { + try { + cb.await(); + } catch (InterruptedException ex) { + throw new RuntimeException(ex); + } catch (BrokenBarrierException ex) { + throw new RuntimeException(ex); + } + } + @Test + public void casBasedUnsafe() { + if (!UnsafeAccess.isUnsafeAvailable()) { + return; + } + long offset = UnsafeAccess.addressOf(IntField.class, "value"); + IntField f = new IntField(); + + assertTrue(UnsafeAccess.compareAndSwapInt(f, offset, 0, 1)); + assertFalse(UnsafeAccess.compareAndSwapInt(f, offset, 0, 2)); + + assertEquals(1, UnsafeAccess.getAndAddInt(f, offset, 2)); + + assertEquals(3, UnsafeAccess.getAndIncrementInt(f, offset)); + + assertEquals(4, UnsafeAccess.getAndSetInt(f, offset, 0)); + } + + @Test + public void powerOfTwo() { + assertTrue(Pow2.isPowerOfTwo(1)); + assertTrue(Pow2.isPowerOfTwo(2)); + assertFalse(Pow2.isPowerOfTwo(3)); + assertTrue(Pow2.isPowerOfTwo(4)); + assertFalse(Pow2.isPowerOfTwo(5)); + assertTrue(Pow2.isPowerOfTwo(8)); + assertFalse(Pow2.isPowerOfTwo(13)); + assertTrue(Pow2.isPowerOfTwo(16)); + assertFalse(Pow2.isPowerOfTwo(25)); + assertFalse(Pow2.isPowerOfTwo(31)); + assertTrue(Pow2.isPowerOfTwo(32)); + } + + @Test(expected = NullPointerException.class) + public void testMpmcArrayQueueNull() { + if (!UnsafeAccess.isUnsafeAvailable()) { + return; + } + MpmcArrayQueue q = new MpmcArrayQueue(16); + q.offer(null); + } + + @Test(expected = UnsupportedOperationException.class) + public void testMpmcArrayQueueIterator() { + if (!UnsafeAccess.isUnsafeAvailable()) { + return; + } + MpmcArrayQueue q = new MpmcArrayQueue(16); + q.iterator(); + } + + @Test + public void testMpmcArrayQueueOfferPoll() { + if (!UnsafeAccess.isUnsafeAvailable()) { + return; + } + Queue q = new MpmcArrayQueue(128); + + testOfferPoll(q); + } + @Test public void testMpmcOfferUpToCapacity() { + if (!UnsafeAccess.isUnsafeAvailable()) { + return; + } int n = 128; MpmcArrayQueue queue = new MpmcArrayQueue(n); for (int i = 0; i < n; i++) { @@ -31,22 +112,352 @@ public void testMpmcOfferUpToCapacity() { } assertFalse(queue.offer(n)); } + @Test(expected = UnsupportedOperationException.class) + public void testMpscLinkedAtomicQueueIterator() { + MpscLinkedAtomicQueue q = new MpscLinkedAtomicQueue(); + q.iterator(); + } + + @Test(expected = NullPointerException.class) + public void testMpscLinkedAtomicQueueNull() { + MpscLinkedAtomicQueue q = new MpscLinkedAtomicQueue(); + q.offer(null); + } + @Test - public void testSpscOfferUpToCapacity() { + public void testMpscLinkedAtomicQueueOfferPoll() { + MpscLinkedAtomicQueue q = new MpscLinkedAtomicQueue(); + + testOfferPoll(q); + } + + @Test(timeout = 2000) + public void testMpscLinkedAtomicQueuePipelined() throws InterruptedException { + final MpscLinkedAtomicQueue q = new MpscLinkedAtomicQueue(); + + Set set = new HashSet(); + for (int i = 0; i < 1000 * 1000; i++) { + set.add(i); + } + + final CyclicBarrier cb = new CyclicBarrier(3); + + Thread t1 = new Thread(new Runnable() { + @Override + public void run() { + await(cb); + for (int i = 0; i < 500 * 1000; i++) { + q.offer(i); + } + } + }); + Thread t2 = new Thread(new Runnable() { + @Override + public void run() { + await(cb); + for (int i = 500 * 1000; i < 1000 * 1000; i++) { + q.offer(i); + } + } + }); + + t1.start(); + t2.start(); + + await(cb); + + Integer j; + for (int i = 0; i < 1000 * 1000; i++) { + while ((j = q.poll()) == null); + assertTrue("Value " + j + " already removed", set.remove(j)); + } + assertTrue("Set is not empty", set.isEmpty()); + } + + @Test(expected = UnsupportedOperationException.class) + public void testMpscLinkedQueueIterator() { + if (!UnsafeAccess.isUnsafeAvailable()) { + return; + } + MpscLinkedQueue q = new MpscLinkedQueue(); + q.iterator(); + } + + @Test(expected = NullPointerException.class) + public void testMpscLinkedQueueNull() { + if (!UnsafeAccess.isUnsafeAvailable()) { + return; + } + MpscLinkedQueue q = new MpscLinkedQueue(); + q.offer(null); + } + + @Test + public void testMpscLinkedQueueOfferPoll() { + if (!UnsafeAccess.isUnsafeAvailable()) { + return; + } + MpscLinkedQueue q = new MpscLinkedQueue(); + + testOfferPoll(q); + } + @Test(timeout = 2000) + public void testMpscLinkedQueuePipelined() throws InterruptedException { + if (!UnsafeAccess.isUnsafeAvailable()) { + return; + } + final MpscLinkedQueue q = new MpscLinkedQueue(); + + Set set = new HashSet(); + for (int i = 0; i < 1000 * 1000; i++) { + set.add(i); + } + + final CyclicBarrier cb = new CyclicBarrier(3); + + Thread t1 = new Thread(new Runnable() { + @Override + public void run() { + await(cb); + for (int i = 0; i < 500 * 1000; i++) { + q.offer(i); + } + } + }); + Thread t2 = new Thread(new Runnable() { + @Override + public void run() { + await(cb); + for (int i = 500 * 1000; i < 1000 * 1000; i++) { + q.offer(i); + } + } + }); + + t1.start(); + t2.start(); + + await(cb); + + Integer j; + for (int i = 0; i < 1000 * 1000; i++) { + while ((j = q.poll()) == null); + assertTrue("Value " + j + " already removed", set.remove(j)); + } + assertTrue("Set is not empty", set.isEmpty()); + } + + protected void testOfferPoll(Queue q) { + for (int i = 0; i < 64; i++) { + assertTrue(q.offer(i)); + } + assertFalse(q.isEmpty()); + for (int i = 0; i < 64; i++) { + assertEquals((Integer)i, q.peek()); + + assertEquals(64 - i, q.size()); + + assertEquals((Integer)i, q.poll()); + } + assertTrue(q.isEmpty()); + + for (int i = 0; i < 64; i++) { + assertTrue(q.offer(i)); + assertEquals((Integer)i, q.poll()); + } + + assertTrue(q.isEmpty()); + assertNull(q.peek()); + assertNull(q.poll()); + } + + @Test(expected = NullPointerException.class) + public void testSpmcArrayQueueNull() { + if (!UnsafeAccess.isUnsafeAvailable()) { + return; + } + SpmcArrayQueue q = new SpmcArrayQueue(16); + q.offer(null); + } + + @Test + public void testSpmcArrayQueueOfferPoll() { + if (!UnsafeAccess.isUnsafeAvailable()) { + return; + } + Queue q = new SpmcArrayQueue(128); + + testOfferPoll(q); + } + @Test(expected = UnsupportedOperationException.class) + public void testSpmcArrayQueueIterator() { + if (!UnsafeAccess.isUnsafeAvailable()) { + return; + } + SpmcArrayQueue q = new SpmcArrayQueue(16); + q.iterator(); + } + + @Test + public void testSpmcOfferUpToCapacity() { + if (!UnsafeAccess.isUnsafeAvailable()) { + return; + } int n = 128; - SpscArrayQueue queue = new SpscArrayQueue(n); + SpmcArrayQueue queue = new SpmcArrayQueue(n); for (int i = 0; i < n; i++) { assertTrue(queue.offer(i)); } assertFalse(queue.offer(n)); } + + @Test(expected = NullPointerException.class) + public void testSpscArrayQueueNull() { + if (!UnsafeAccess.isUnsafeAvailable()) { + return; + } + SpscArrayQueue q = new SpscArrayQueue(16); + q.offer(null); + } + @Test - public void testSpmcOfferUpToCapacity() { + public void testSpscArrayQueueOfferPoll() { + if (!UnsafeAccess.isUnsafeAvailable()) { + return; + } + Queue q = new SpscArrayQueue(128); + + testOfferPoll(q); + } + @Test(expected = UnsupportedOperationException.class) + public void testSpscArrayQueueIterator() { + if (!UnsafeAccess.isUnsafeAvailable()) { + return; + } + SpscArrayQueue q = new SpscArrayQueue(16); + q.iterator(); + } + @Test(expected = UnsupportedOperationException.class) + public void testSpscLinkedAtomicQueueIterator() { + SpscLinkedAtomicQueue q = new SpscLinkedAtomicQueue(); + q.iterator(); + } + @Test(expected = NullPointerException.class) + public void testSpscLinkedAtomicQueueNull() { + SpscLinkedAtomicQueue q = new SpscLinkedAtomicQueue(); + q.offer(null); + } + + @Test + public void testSpscLinkedAtomicQueueOfferPoll() { + SpscLinkedAtomicQueue q = new SpscLinkedAtomicQueue(); + + testOfferPoll(q); + } + + @Test(timeout = 2000) + public void testSpscLinkedAtomicQueuePipelined() throws InterruptedException { + final SpscLinkedAtomicQueue q = new SpscLinkedAtomicQueue(); + final AtomicInteger count = new AtomicInteger(); + + Thread t = new Thread(new Runnable() { + @Override + public void run() { + Integer j; + for (int i = 0; i < 1000 * 1000; i++) { + while ((j = q.poll()) == null); + if (j == i) { + count.getAndIncrement(); + } + } + } + }); + t.start(); + + for (int i = 0; i < 1000 * 1000; i++) { + assertTrue(q.offer(i)); + } + t.join(); + + assertEquals(1000 * 1000, count.get()); + } + + @Test(expected = UnsupportedOperationException.class) + public void testSpscLinkedQueueIterator() { + if (!UnsafeAccess.isUnsafeAvailable()) { + return; + } + SpscLinkedQueue q = new SpscLinkedQueue(); + q.iterator(); + } + + @Test(expected = NullPointerException.class) + public void testSpscLinkedQueueNull() { + if (!UnsafeAccess.isUnsafeAvailable()) { + return; + } + SpscLinkedQueue q = new SpscLinkedQueue(); + q.offer(null); + } + + @Test + public void testSpscLinkedQueueOfferPoll() { + if (!UnsafeAccess.isUnsafeAvailable()) { + return; + } + SpscLinkedQueue q = new SpscLinkedQueue(); + + testOfferPoll(q); + } + + @Test(timeout = 2000) + public void testSpscLinkedQueuePipelined() throws InterruptedException { + if (!UnsafeAccess.isUnsafeAvailable()) { + return; + } + final SpscLinkedQueue q = new SpscLinkedQueue(); + final AtomicInteger count = new AtomicInteger(); + + Thread t = new Thread(new Runnable() { + @Override + public void run() { + Integer j; + for (int i = 0; i < 1000 * 1000; i++) { + while ((j = q.poll()) == null); + if (j == i) { + count.getAndIncrement(); + } + } + } + }); + t.start(); + + for (int i = 0; i < 1000 * 1000; i++) { + assertTrue(q.offer(i)); + } + t.join(); + + assertEquals(1000 * 1000, count.get()); + } + + @Test + public void testSpscOfferUpToCapacity() { + if (!UnsafeAccess.isUnsafeAvailable()) { + return; + } int n = 128; - SpmcArrayQueue queue = new SpmcArrayQueue(n); + SpscArrayQueue queue = new SpscArrayQueue(n); for (int i = 0; i < n; i++) { assertTrue(queue.offer(i)); } assertFalse(queue.offer(n)); } + + @Test(expected = InternalError.class) + public void testUnsafeAccessAddressOf() { + if (!UnsafeAccess.isUnsafeAvailable()) { + return; + } + UnsafeAccess.addressOf(Object.class, "field"); + } } From 1aab0f744f62c14356e3dcc29e2b2cdcada6cfbc Mon Sep 17 00:00:00 2001 From: akarnokd Date: Fri, 24 Jul 2015 08:54:37 +0200 Subject: [PATCH 17/81] Fix take swallowing exception if thrown by the exactly the nth onNext call to it. --- .../rx/internal/operators/OperatorTake.java | 26 ++++++---- .../internal/operators/OperatorTakeTest.java | 49 ++++++++++--------- 2 files changed, 43 insertions(+), 32 deletions(-) diff --git a/src/main/java/rx/internal/operators/OperatorTake.java b/src/main/java/rx/internal/operators/OperatorTake.java index 0cc42b88ef..31811537b5 100644 --- a/src/main/java/rx/internal/operators/OperatorTake.java +++ b/src/main/java/rx/internal/operators/OperatorTake.java @@ -43,12 +43,13 @@ public OperatorTake(int limit) { public Subscriber call(final Subscriber child) { final Subscriber parent = new Subscriber() { - int count = 0; - boolean completed = false; + int count; + boolean completed; @Override public void onCompleted() { if (!completed) { + completed = true; child.onCompleted(); } } @@ -56,20 +57,27 @@ public void onCompleted() { @Override public void onError(Throwable e) { if (!completed) { - child.onError(e); + completed = true; + try { + child.onError(e); + } finally { + unsubscribe(); + } } } @Override public void onNext(T i) { if (!isUnsubscribed()) { - if (++count >= limit) { - completed = true; - } + boolean stop = ++count >= limit; child.onNext(i); - if (completed) { - child.onCompleted(); - unsubscribe(); + if (stop && !completed) { + completed = true; + try { + child.onCompleted(); + } finally { + unsubscribe(); + } } } } diff --git a/src/test/java/rx/internal/operators/OperatorTakeTest.java b/src/test/java/rx/internal/operators/OperatorTakeTest.java index 111eb6abbd..3384445d5b 100644 --- a/src/test/java/rx/internal/operators/OperatorTakeTest.java +++ b/src/test/java/rx/internal/operators/OperatorTakeTest.java @@ -16,36 +16,21 @@ package rx.internal.operators; import static org.junit.Assert.*; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyString; -import static org.mockito.Mockito.inOrder; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Matchers.*; +import static org.mockito.Mockito.*; import java.util.Arrays; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.*; +import java.util.concurrent.atomic.*; import org.junit.Test; import org.mockito.InOrder; -import rx.Observable; +import rx.*; import rx.Observable.OnSubscribe; -import rx.Observer; -import rx.Producer; -import rx.Subscriber; -import rx.Subscription; -import rx.functions.Action1; -import rx.functions.Func1; -import rx.observers.Subscribers; -import rx.observers.TestSubscriber; +import rx.exceptions.TestException; +import rx.functions.*; +import rx.observers.*; import rx.schedulers.Schedulers; public class OperatorTakeTest { @@ -414,4 +399,22 @@ public void call(Long n) { ts.assertNoErrors(); assertEquals(2,requests.get()); } + + @Test + public void takeFinalValueThrows() { + Observable source = Observable.just(1).take(1); + + TestSubscriber ts = new TestSubscriber() { + @Override + public void onNext(Integer t) { + throw new TestException(); + } + }; + + source.subscribe(ts); + + ts.assertNoValues(); + ts.assertError(TestException.class); + ts.assertNotCompleted(); + } } From 2f815f7eb08f1e5d09b024ca9dc4c567ebc8d986 Mon Sep 17 00:00:00 2001 From: akarnokd Date: Fri, 24 Jul 2015 12:32:29 +0200 Subject: [PATCH 18/81] Test coverage of rx.functions utility methods. --- src/test/java/rx/functions/ActionsTest.java | 290 ++++++++++++++++++ src/test/java/rx/functions/FunctionsTest.java | 268 ++++++++++++++++ 2 files changed, 558 insertions(+) create mode 100644 src/test/java/rx/functions/ActionsTest.java create mode 100644 src/test/java/rx/functions/FunctionsTest.java diff --git a/src/test/java/rx/functions/ActionsTest.java b/src/test/java/rx/functions/ActionsTest.java new file mode 100644 index 0000000000..8ffecb97ca --- /dev/null +++ b/src/test/java/rx/functions/ActionsTest.java @@ -0,0 +1,290 @@ +/** + * Copyright 2014 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.functions; + +import static org.junit.Assert.*; + +import java.lang.reflect.*; +import java.util.Arrays; +import java.util.concurrent.atomic.AtomicLong; + +import org.junit.Test; + +public class ActionsTest { + + @Test + public void testEmptyArities() { + Action0 a0 = Actions.empty(); + a0.call(); + + Action1 a1 = Actions.empty(); + a1.call(1); + + Action2 a2 = Actions.empty(); + a2.call(1, 2); + + Action3 a3 = Actions.empty(); + a3.call(1, 2, 3); + + Action4 a4 = Actions.empty(); + a4.call(1, 2, 3, 4); + + Action5 a5 = Actions.empty(); + a5.call(1, 2, 3, 4, 5); + + Action6 a6 = Actions.empty(); + a6.call(1, 2, 3, 4, 5, 6); + + Action7 a7 = Actions.empty(); + a7.call(1, 2, 3, 4, 5, 6, 7); + + Action8 a8 = Actions.empty(); + a8.call(1, 2, 3, 4, 5, 6, 7, 8); + + Action9 a9 = Actions.empty(); + a9.call(1, 2, 3, 4, 5, 6, 7, 8, 9); + + ActionN an0 = Actions.empty(); + an0.call(); + + ActionN an1 = Actions.empty(); + an1.call(1); + + ActionN ann = Actions.empty(); + ann.call(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + + ActionN annn = Actions.empty(); + annn.call(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, + 11, 12, 13, 14, 15, 16, 17, 18, 19, 20); + } + + @Test + public void testToFunc0() { + final AtomicLong value = new AtomicLong(-1L); + final Action0 action = new Action0() { + @Override + public void call() { + value.set(0); + } + }; + + assertNull(Actions.toFunc(action).call()); + assertEquals(0, value.get()); + value.set(-1L); + assertEquals((Integer)0, Actions.toFunc(action, 0).call()); + assertEquals(0, value.get()); + } + + @Test + public void testToFunc1() { + final AtomicLong value = new AtomicLong(-1L); + final Action1 action = new Action1() { + @Override + public void call(Integer t1) { + value.set(t1); + } + }; + + assertNull(Actions.toFunc(action).call(1)); + assertEquals(1, value.get()); + value.set(-1L); + assertEquals((Integer)0, Actions.toFunc(action, 0).call(1)); + assertEquals(1, value.get()); + } + + @Test + public void testToFunc2() { + final AtomicLong value = new AtomicLong(-1L); + final Action2 action = new Action2() { + @Override + public void call(Integer t1, Integer t2) { + value.set(t1 | t2); + } + }; + + assertNull(Actions.toFunc(action).call(1, 2)); + assertNull(Actions.toFunc(action).call(1, 2)); + assertEquals(3, value.get()); + value.set(-1L); + assertEquals((Integer)0, Actions.toFunc(action, 0).call(1, 2)); + assertEquals(3, value.get()); + } + + @Test + public void testToFunc3() { + final AtomicLong value = new AtomicLong(-1L); + final Action3 action = new Action3() { + @Override + public void call(Integer t1, Integer t2, Integer t3) { + value.set(t1 | t2 | t3); + } + }; + + assertNull(Actions.toFunc(action).call(1, 2, 4)); + assertEquals(7, value.get()); + value.set(-1L); + assertEquals((Integer)0, Actions.toFunc(action, 0).call(1, 2, 4)); + assertEquals(7, value.get()); + } + + @Test + public void testToFunc4() { + final AtomicLong value = new AtomicLong(-1L); + final Action4 action = new Action4() { + @Override + public void call(Integer t1, Integer t2, Integer t3, Integer t4) { + value.set(t1 | t2 | t3 | t4); + } + }; + + assertNull(Actions.toFunc(action).call(1, 2, 4, 8)); + assertEquals(15, value.get()); + value.set(-1L); + assertEquals((Integer)0, Actions.toFunc(action, 0).call(1, 2, 4, 8)); + assertEquals(15, value.get()); + } + + @Test + public void testToFunc5() { + final AtomicLong value = new AtomicLong(-1L); + final Action5 action = + new Action5() { + @Override + public void call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5) { + value.set(t1 | t2 | t3 | t4 | t5); + } + }; + + assertNull(Actions.toFunc(action).call(1, 2, 4, 8, 16)); + assertEquals(31, value.get()); + value.set(-1L); + assertEquals((Integer)0, Actions.toFunc(action, 0).call(1, 2, 4, 8, 16)); + assertEquals(31, value.get()); + } + + @Test + public void testToFunc6() { + final AtomicLong value = new AtomicLong(-1L); + final Action6 action = + new Action6() { + @Override + public void call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6) { + value.set(t1 | t2 | t3 | t4 | t5 | t6); + } + }; + + assertNull(Actions.toFunc(action).call(1, 2, 4, 8, 16, 32)); + assertEquals(63, value.get()); + value.set(-1L); + assertEquals((Integer)0, Actions.toFunc(action, 0).call(1, 2, 4, 8, 16, 32)); + assertEquals(63, value.get()); + } + + @Test + public void testToFunc7() { + final AtomicLong value = new AtomicLong(-1L); + final Action7 action = + new Action7() { + @Override + public void call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6, Integer t7) { + value.set(t1 | t2 | t3 | t4 | t5 | t6 | t7); + } + }; + + assertNull(Actions.toFunc(action).call(1, 2, 4, 8, 16, 32, 64)); + assertEquals(127, value.get()); + value.set(-1L); + assertEquals((Integer)0, Actions.toFunc(action, 0).call(1, 2, 4, 8, 16, 32, 64)); + assertEquals(127, value.get()); + } + @Test + public void testToFunc8() { + final AtomicLong value = new AtomicLong(-1L); + final Action8 action = + new Action8() { + @Override + public void call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6, Integer t7, Integer t8) { + value.set(t1 | t2 | t3 | t4 | t5 | t6 | t7 | t8); + } + }; + + assertNull(Actions.toFunc(action).call(1, 2, 4, 8, 16, 32, 64, 128)); + assertEquals(255, value.get()); + value.set(-1L); + assertEquals((Integer)0, Actions.toFunc(action, 0).call(1, 2, 4, 8, 16, 32, 64, 128)); + assertEquals(255, value.get()); + } + @Test + public void testToFunc9() { + final AtomicLong value = new AtomicLong(-1L); + final Action9 action = + new Action9() { + @Override + public void call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6, Integer t7, Integer t8, Integer t9) { + value.set(t1 | t2 | t3 | t4 | t5 | t6 | t7 | t8 | t9); + } + }; + + assertNull(Actions.toFunc(action).call(1, 2, 4, 8, 16, 32, 64, 128, 256)); + assertEquals(511, value.get()); + value.set(-1L); + assertEquals((Integer)0, Actions.toFunc(action, 0).call(1, 2, 4, 8, 16, 32, 64, 128, 256)); + assertEquals(511, value.get()); + } + + @Test + public void testToFuncN() { + for (int i = 0; i < 100; i++) { + final AtomicLong value = new AtomicLong(-1L); + final ActionN action = new ActionN() { + @Override + public void call(Object... args) { + int sum = 0; + for (Object o : args) { + sum += (Integer)o; + } + value.set(sum); + } + }; + Object[] arr = new Object[i]; + Arrays.fill(arr, 1); + + assertNull(Actions.toFunc(action).call(arr)); + assertEquals(i, value.get()); + value.set(-1L); + assertEquals((Integer)0, Actions.toFunc(action, 0).call(arr)); + assertEquals(i, value.get()); + } + } + + @Test + public void testNotInstantiable() { + try { + Constructor c = Actions.class.getDeclaredConstructor(); + c.setAccessible(true); + Object instance = c.newInstance(); + fail("Could instantiate Actions! " + instance); + } catch (NoSuchMethodException ex) { + ex.printStackTrace(); + } catch (InvocationTargetException ex) { + ex.printStackTrace(); + } catch (InstantiationException ex) { + ex.printStackTrace(); + } catch (IllegalAccessException ex) { + ex.printStackTrace(); + } + } +} diff --git a/src/test/java/rx/functions/FunctionsTest.java b/src/test/java/rx/functions/FunctionsTest.java new file mode 100644 index 0000000000..1d2cc95374 --- /dev/null +++ b/src/test/java/rx/functions/FunctionsTest.java @@ -0,0 +1,268 @@ +/** + * Copyright 2014 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.functions; + +import static org.junit.Assert.*; + +import java.lang.reflect.*; +import java.util.Arrays; +import java.util.concurrent.atomic.AtomicLong; + +import org.junit.Test; + +public class FunctionsTest { + @Test + public void testNotInstantiable() { + try { + Constructor c = Functions.class.getDeclaredConstructor(); + c.setAccessible(true); + Object instance = c.newInstance(); + fail("Could instantiate Actions! " + instance); + } catch (NoSuchMethodException ex) { + ex.printStackTrace(); + } catch (InvocationTargetException ex) { + ex.printStackTrace(); + } catch (InstantiationException ex) { + ex.printStackTrace(); + } catch (IllegalAccessException ex) { + ex.printStackTrace(); + } + } + + @Test(expected = RuntimeException.class) + public void testFromFunc0() { + Func0 func = new Func0() { + @Override + public Integer call() { + return 0; + } + }; + + Object[] params = new Object[0]; + assertEquals((Integer)0, Functions.fromFunc(func).call(params)); + + Functions.fromFunc(func).call(Arrays.copyOf(params, params.length + 1)); + } + + @Test(expected = RuntimeException.class) + public void testFromFunc1() { + Func1 func = new Func1() { + @Override + public Integer call(Integer t1) { + return t1; + } + }; + + Object[] params = new Object[] { 1 }; + assertEquals((Integer)1, Functions.fromFunc(func).call(params)); + + Functions.fromFunc(func).call(Arrays.copyOf(params, params.length + 1)); + } + + @Test(expected = RuntimeException.class) + public void testFromFunc2() { + Func2 func = new Func2() { + @Override + public Integer call(Integer t1, Integer t2) { + return t1 | t2; + } + }; + + Object[] params = new Object[] { 1, 2 }; + assertEquals((Integer)3, Functions.fromFunc(func).call(params)); + + Functions.fromFunc(func).call(Arrays.copyOf(params, params.length + 1)); + } + + @Test(expected = RuntimeException.class) + public void testFromFunc3() { + Func3 func = new Func3() { + @Override + public Integer call(Integer t1, Integer t2, Integer t3) { + return t1 | t2 | t3; + } + }; + + Object[] params = new Object[] { 1, 2, 4 }; + assertEquals((Integer)7, Functions.fromFunc(func).call(params)); + + Functions.fromFunc(func).call(Arrays.copyOf(params, params.length + 1)); + } + + @Test(expected = RuntimeException.class) + public void testFromFunc4() { + Func4 func = + new Func4() { + @Override + public Integer call(Integer t1, Integer t2, Integer t3, Integer t4) { + return t1 | t2 | t3 | t4; + } + }; + + Object[] params = new Object[] { 1, 2, 4, 8 }; + assertEquals((Integer)15, Functions.fromFunc(func).call(params)); + + Functions.fromFunc(func).call(Arrays.copyOf(params, params.length + 1)); + } + + @Test(expected = RuntimeException.class) + public void testFromFunc5() { + Func5 func = + new Func5() { + @Override + public Integer call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5) { + return t1 | t2 | t3 | t4 | t5; + } + }; + + Object[] params = new Object[] { 1, 2, 4, 8, 16 }; + assertEquals((Integer)31, Functions.fromFunc(func).call(params)); + + Functions.fromFunc(func).call(Arrays.copyOf(params, params.length + 1)); + } + + @Test(expected = RuntimeException.class) + public void testFromFunc6() { + Func6 func = + new Func6() { + @Override + public Integer call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6) { + return t1 | t2 | t3 | t4 | t5 | t6; + } + }; + + Object[] params = new Object[] { 1, 2, 4, 8, 16, 32 }; + assertEquals((Integer)63, Functions.fromFunc(func).call(params)); + + Functions.fromFunc(func).call(Arrays.copyOf(params, params.length + 1)); + } + + @Test(expected = RuntimeException.class) + public void testFromFunc7() { + Func7 func = + new Func7() { + @Override + public Integer call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6, Integer t7) { + return t1 | t2 | t3 | t4 | t5 | t6 | t7; + } + }; + + Object[] params = new Object[] { 1, 2, 4, 8, 16, 32, 64 }; + assertEquals((Integer)127, Functions.fromFunc(func).call(params)); + + Functions.fromFunc(func).call(Arrays.copyOf(params, params.length + 1)); + } + + @Test(expected = RuntimeException.class) + public void testFromFunc8() { + Func8 func = + new Func8() { + @Override + public Integer call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6, Integer t7, Integer t8) { + return t1 | t2 | t3 | t4 | t5 | t6 | t7 | t8; + } + }; + + Object[] params = new Object[] { 1, 2, 4, 8, 16, 32, 64, 128 }; + assertEquals((Integer)255, Functions.fromFunc(func).call(params)); + + Functions.fromFunc(func).call(Arrays.copyOf(params, params.length + 1)); + } + + @Test(expected = RuntimeException.class) + public void testFromFunc9() { + Func9 func = + new Func9() { + @Override + public Integer call(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, Integer t6, Integer t7, Integer t8, Integer t9) { + return t1 | t2 | t3 | t4 | t5 | t6 | t7 | t8 | t9; + } + }; + + Object[] params = new Object[] { 1, 2, 4, 8, 16, 32, 64, 128, 256 }; + assertEquals((Integer)511, Functions.fromFunc(func).call(params)); + + Functions.fromFunc(func).call(Arrays.copyOf(params, params.length + 1)); + } + + @Test(expected = RuntimeException.class) + public void testFromAction0() { + final AtomicLong value = new AtomicLong(); + Action0 action = new Action0() { + @Override + public void call() { + value.set(0); + } + }; + + Object[] params = new Object[] { }; + Functions.fromAction(action).call(params); + assertEquals(0, value.get()); + + Functions.fromAction(action).call(Arrays.copyOf(params, params.length + 1)); + } + + @Test(expected = RuntimeException.class) + public void testFromAction1() { + final AtomicLong value = new AtomicLong(); + Action1 action = new Action1() { + @Override + public void call(Integer t1) { + value.set(t1); + } + }; + + Object[] params = new Object[] { 1 }; + Functions.fromAction(action).call(params); + assertEquals(1, value.get()); + + Functions.fromAction(action).call(Arrays.copyOf(params, params.length + 1)); + } + + @Test(expected = RuntimeException.class) + public void testFromAction2() { + final AtomicLong value = new AtomicLong(); + Action2 action = new Action2() { + @Override + public void call(Integer t1, Integer t2) { + value.set(t1 | t2); + } + }; + + Object[] params = new Object[] { 1, 2 }; + Functions.fromAction(action).call(params); + assertEquals(3, value.get()); + + Functions.fromAction(action).call(Arrays.copyOf(params, params.length + 1)); + } + + @Test(expected = RuntimeException.class) + public void testFromAction3() { + final AtomicLong value = new AtomicLong(); + Action3 action = new Action3() { + @Override + public void call(Integer t1, Integer t2, Integer t3) { + value.set(t1 | t2 | t3); + } + }; + + Object[] params = new Object[] { 1, 2, 4 }; + Functions.fromAction(action).call(params); + assertEquals(7, value.get()); + + Functions.fromAction(action).call(Arrays.copyOf(params, params.length + 1)); + } +} From 36167ffdf36e86355307a519471edd11d1c106ae Mon Sep 17 00:00:00 2001 From: akarnokd Date: Mon, 27 Jul 2015 08:23:56 +0200 Subject: [PATCH 19/81] cache() now supports backpressure (again) --- src/main/java/rx/Observable.java | 4 +- .../internal/operators/CachedObservable.java | 462 ++++++++++++++++++ .../internal/operators/OnSubscribeCache.java | 76 --- .../rx/internal/util/LinkedArrayList.java | 136 ++++++ .../rx/internal/util/LinkedArrayListTest.java | 37 ++ .../operators/CachedObservableTest.java | 293 +++++++++++ .../operators/OnSubscribeCacheTest.java | 164 ------- 7 files changed, 930 insertions(+), 242 deletions(-) create mode 100644 src/main/java/rx/internal/operators/CachedObservable.java delete mode 100644 src/main/java/rx/internal/operators/OnSubscribeCache.java create mode 100644 src/main/java/rx/internal/util/LinkedArrayList.java create mode 100644 src/main/java/rx/internal/util/LinkedArrayListTest.java create mode 100644 src/test/java/rx/internal/operators/CachedObservableTest.java delete mode 100644 src/test/java/rx/internal/operators/OnSubscribeCacheTest.java diff --git a/src/main/java/rx/Observable.java b/src/main/java/rx/Observable.java index 9fd476eab7..a6aabc082e 100644 --- a/src/main/java/rx/Observable.java +++ b/src/main/java/rx/Observable.java @@ -3599,7 +3599,7 @@ public final Observable> buffer(Observable boundary, int initialC * @see ReactiveX operators documentation: Replay */ public final Observable cache() { - return create(new OnSubscribeCache(this)); + return CachedObservable.from(this); } /** @@ -3634,7 +3634,7 @@ public final Observable cache() { * @see ReactiveX operators documentation: Replay */ public final Observable cache(int capacityHint) { - return create(new OnSubscribeCache(this, capacityHint)); + return CachedObservable.from(this, capacityHint); } /** diff --git a/src/main/java/rx/internal/operators/CachedObservable.java b/src/main/java/rx/internal/operators/CachedObservable.java new file mode 100644 index 0000000000..0231c3590f --- /dev/null +++ b/src/main/java/rx/internal/operators/CachedObservable.java @@ -0,0 +1,462 @@ +/** + * Copyright 2014 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.internal.operators; + +import java.util.concurrent.atomic.*; + +import rx.*; +import rx.exceptions.*; +import rx.internal.util.LinkedArrayList; +import rx.subscriptions.SerialSubscription; + +/** + * An observable which auto-connects to another observable, caches the elements + * from that observable but allows terminating the connection and completing the cache. + * + * @param the source element type + */ +public final class CachedObservable extends Observable { + /** The cache and replay state. */ + private CacheState state; + + /** + * Creates a cached Observable with a default capacity hint of 16. + * @param source the source Observable to cache + * @return the CachedObservable instance + */ + public static CachedObservable from(Observable source) { + return from(source, 16); + } + + /** + * Creates a cached Observable with the given capacity hint. + * @param source the source Observable to cache + * @param capacityHint the hint for the internal buffer size + * @return the CachedObservable instance + */ + public static CachedObservable from(Observable source, int capacityHint) { + if (capacityHint < 1) { + throw new IllegalArgumentException("capacityHint > 0 required"); + } + CacheState state = new CacheState(source, capacityHint); + CachedSubscribe onSubscribe = new CachedSubscribe(state); + return new CachedObservable(onSubscribe, state); + } + + /** + * Private constructor because state needs to be shared between the Observable body and + * the onSubscribe function. + * @param onSubscribe + * @param state + */ + private CachedObservable(OnSubscribe onSubscribe, CacheState state) { + super(onSubscribe); + this.state = state; + } + + /** + * Check if this cached observable is connected to its source. + * @return true if already connected + */ + /* public */boolean isConnected() { + return state.isConnected; + } + + /** + * Returns true if there are observers subscribed to this observable. + * @return + */ + /* public */ boolean hasObservers() { + return state.producers.length != 0; + } + + /** + * Returns the number of events currently cached. + * @return + */ + /* public */ int cachedEventCount() { + return state.size(); + } + + /** + * Contains the active child producers and the values to replay. + * + * @param + */ + static final class CacheState extends LinkedArrayList implements Observer { + /** The source observable to connect to. */ + final Observable source; + /** Holds onto the subscriber connected to source. */ + final SerialSubscription connection; + /** Guarded by connection (not this). */ + volatile ReplayProducer[] producers; + /** The default empty array of producers. */ + static final ReplayProducer[] EMPTY = new ReplayProducer[0]; + + final NotificationLite nl; + + /** Set to true after connection. */ + volatile boolean isConnected; + /** + * Indicates that the source has completed emitting values or the + * Observable was forcefully terminated. + */ + boolean sourceDone; + + public CacheState(Observable source, int capacityHint) { + super(capacityHint); + this.source = source; + this.producers = EMPTY; + this.nl = NotificationLite.instance(); + this.connection = new SerialSubscription(); + } + /** + * Adds a ReplayProducer to the producers array atomically. + * @param p + */ + public void addProducer(ReplayProducer p) { + // guarding by connection to save on allocating another object + // thus there are two distinct locks guarding the value-addition and child come-and-go + synchronized (connection) { + ReplayProducer[] a = producers; + int n = a.length; + ReplayProducer[] b = new ReplayProducer[n + 1]; + System.arraycopy(a, 0, b, 0, n); + b[n] = p; + producers = b; + } + } + /** + * Removes the ReplayProducer (if present) from the producers array atomically. + * @param p + */ + public void removeProducer(ReplayProducer p) { + synchronized (connection) { + ReplayProducer[] a = producers; + int n = a.length; + int j = -1; + for (int i = 0; i < n; i++) { + if (a[i].equals(p)) { + j = i; + break; + } + } + if (j < 0) { + return; + } + if (n == 1) { + producers = EMPTY; + return; + } + ReplayProducer[] b = new ReplayProducer[n - 1]; + System.arraycopy(a, 0, b, 0, j); + System.arraycopy(a, j + 1, b, j, n - j - 1); + producers = b; + } + } + /** + * Connects the cache to the source. + * Make sure this is called only once. + */ + public void connect() { + Subscriber subscriber = new Subscriber() { + @Override + public void onNext(T t) { + CacheState.this.onNext(t); + } + @Override + public void onError(Throwable e) { + CacheState.this.onError(e); + } + @Override + public void onCompleted() { + CacheState.this.onCompleted(); + } + }; + connection.set(subscriber); + source.unsafeSubscribe(subscriber); + isConnected = true; + } + @Override + public void onNext(T t) { + if (!sourceDone) { + Object o = nl.next(t); + add(o); + dispatch(); + } + } + @Override + public void onError(Throwable e) { + if (!sourceDone) { + sourceDone = true; + Object o = nl.error(e); + add(o); + connection.unsubscribe(); + dispatch(); + } + } + @Override + public void onCompleted() { + if (!sourceDone) { + sourceDone = true; + Object o = nl.completed(); + add(o); + connection.unsubscribe(); + dispatch(); + } + } + /** + * Signals all known children there is work to do. + */ + void dispatch() { + ReplayProducer[] a = producers; + for (ReplayProducer rp : a) { + rp.replay(); + } + } + } + + /** + * Manages the subscription of child subscribers by setting up a replay producer and + * performs auto-connection of the very first subscription. + * @param the value type emitted + */ + static final class CachedSubscribe extends AtomicBoolean implements OnSubscribe { + /** */ + private static final long serialVersionUID = -2817751667698696782L; + final CacheState state; + public CachedSubscribe(CacheState state) { + this.state = state; + } + @Override + public void call(Subscriber t) { + // we can connect first because we replay everything anyway + ReplayProducer rp = new ReplayProducer(t, state); + state.addProducer(rp); + + t.add(rp); + t.setProducer(rp); + + // we ensure a single connection here to save an instance field of AtomicBoolean in state. + if (!get() && compareAndSet(false, true)) { + state.connect(); + } + + // no need to call rp.replay() here because the very first request will trigger it anyway + } + } + + /** + * Keeps track of the current request amount and the replay position for a child Subscriber. + * + * @param + */ + static final class ReplayProducer extends AtomicLong implements Producer, Subscription { + /** */ + private static final long serialVersionUID = -2557562030197141021L; + /** The actual child subscriber. */ + final Subscriber child; + /** The cache state object. */ + final CacheState state; + + /** + * Contains the reference to the buffer segment in replay. + * Accessed after reading state.size() and when emitting == true. + */ + Object[] currentBuffer; + /** + * Contains the index into the currentBuffer where the next value is expected. + * Accessed after reading state.size() and when emitting == true. + */ + int currentIndexInBuffer; + /** + * Contains the absolute index up until the values have been replayed so far. + */ + int index; + + /** Indicates there is a replay going on; guarded by this. */ + boolean emitting; + /** Indicates there were some state changes/replay attempts; guarded by this. */ + boolean missed; + + public ReplayProducer(Subscriber child, CacheState state) { + this.child = child; + this.state = state; + } + @Override + public void request(long n) { + for (;;) { + long r = get(); + if (r < 0) { + return; + } + long u = r + n; + if (u < 0) { + u = Long.MAX_VALUE; + } + if (compareAndSet(r, u)) { + replay(); + return; + } + } + } + /** + * Updates the request count to reflect values have been produced. + * @param n + * @return + */ + public long produced(long n) { + return addAndGet(-n); + } + + @Override + public boolean isUnsubscribed() { + return get() < 0; + } + @Override + public void unsubscribe() { + long r = get(); + if (r >= 0) { + r = getAndSet(-1L); // unsubscribed state is negative + if (r >= 0) { + state.removeProducer(this); + } + } + } + + /** + * Continue replaying available values if there are requests for them. + */ + public void replay() { + // make sure there is only a single thread emitting + synchronized (this) { + if (emitting) { + missed = true; + return; + } + emitting = true; + } + boolean skipFinal = false; + try { + final NotificationLite nl = state.nl; + final Subscriber child = this.child; + + for (;;) { + + long r = get(); + + if (r < 0L) { + skipFinal = true; + return; + } + + // read the size, if it is non-zero, we can safely read the head and + // read values up to the given absolute index + int s = state.size(); + if (s != 0) { + Object[] b = currentBuffer; + + // latch onto the very first buffer now that it is available. + if (b == null) { + b = state.head(); + currentBuffer = b; + } + final int n = b.length - 1; + int j = index; + int k = currentIndexInBuffer; + // eagerly emit any terminal event + if (r == 0) { + Object o = b[k]; + if (nl.isCompleted(o)) { + child.onCompleted(); + skipFinal = true; + unsubscribe(); + return; + } else + if (nl.isError(o)) { + child.onError(nl.getError(o)); + skipFinal = true; + unsubscribe(); + return; + } + } else + if (r > 0) { + int valuesProduced = 0; + + while (j < s && r > 0) { + if (child.isUnsubscribed()) { + skipFinal = true; + return; + } + if (k == n) { + b = (Object[])b[n]; + k = 0; + } + Object o = b[k]; + + try { + if (nl.accept(child, o)) { + skipFinal = true; + unsubscribe(); + return; + } + } catch (Throwable err) { + Exceptions.throwIfFatal(err); + skipFinal = true; + unsubscribe(); + if (!nl.isError(o) && !nl.isCompleted(o)) { + child.onError(OnErrorThrowable.addValueAsLastCause(err, nl.getValue(o))); + } + return; + } + + k++; + j++; + r--; + valuesProduced++; + } + + if (child.isUnsubscribed()) { + skipFinal = true; + return; + } + + index = j; + currentIndexInBuffer = k; + currentBuffer = b; + produced(valuesProduced); + } + } + + synchronized (this) { + if (!missed) { + emitting = false; + skipFinal = true; + return; + } + missed = false; + } + } + } finally { + if (!skipFinal) { + synchronized (this) { + emitting = false; + } + } + } + } + } +} \ No newline at end of file diff --git a/src/main/java/rx/internal/operators/OnSubscribeCache.java b/src/main/java/rx/internal/operators/OnSubscribeCache.java deleted file mode 100644 index a568fd0e0b..0000000000 --- a/src/main/java/rx/internal/operators/OnSubscribeCache.java +++ /dev/null @@ -1,76 +0,0 @@ -/** - * Copyright 2014 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.internal.operators; - -import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; - -import rx.Observable; -import rx.Observable.OnSubscribe; -import rx.Subscriber; -import rx.subjects.ReplaySubject; -import rx.subjects.Subject; - -/** - * This method has similar behavior to {@link Observable#replay()} except that this auto-subscribes - * to the source Observable rather than returning a connectable Observable. - *

- * - *

- * This is useful with an Observable that you want to cache responses when you can't control the - * subscribe/unsubscribe behavior of all the Observers. - *

- * Note: You sacrifice the ability to unsubscribe from the origin when you use this operator, so be - * careful not to use this operator on Observables that emit infinite or very large numbers of - * items, as this will use up memory. - * - * @param - * the cached value type - */ -public final class OnSubscribeCache implements OnSubscribe { - protected final Observable source; - protected final Subject cache; - volatile int sourceSubscribed; - @SuppressWarnings("rawtypes") - static final AtomicIntegerFieldUpdater SRC_SUBSCRIBED_UPDATER - = AtomicIntegerFieldUpdater.newUpdater(OnSubscribeCache.class, "sourceSubscribed"); - - public OnSubscribeCache(Observable source) { - this(source, ReplaySubject. create()); - } - - public OnSubscribeCache(Observable source, int capacity) { - this(source, ReplaySubject. create(capacity)); - } - - /* accessible to tests */OnSubscribeCache(Observable source, Subject cache) { - this.source = source; - this.cache = cache; - } - - @Override - public void call(Subscriber s) { - if (SRC_SUBSCRIBED_UPDATER.compareAndSet(this, 0, 1)) { - source.subscribe(cache); - /* - * Note that we will never unsubscribe from 'source' unless we receive `onCompleted` or `onError`, - * as we want to receive and cache all of its values. - * - * This means this should never be used on an infinite or very large sequence, similar to toList(). - */ - } - cache.unsafeSubscribe(s); - } -} diff --git a/src/main/java/rx/internal/util/LinkedArrayList.java b/src/main/java/rx/internal/util/LinkedArrayList.java new file mode 100644 index 0000000000..57a1289640 --- /dev/null +++ b/src/main/java/rx/internal/util/LinkedArrayList.java @@ -0,0 +1,136 @@ +/** + * Copyright 2014 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.internal.util; + +import java.util.*; + +/** + * A list implementation which combines an ArrayList with a LinkedList to + * avoid copying values when the capacity needs to be increased. + *

+ * The class is non final to allow embedding it directly and thus saving on object allocation. + */ +public class LinkedArrayList { + /** The capacity of each array segment. */ + final int capacityHint; + /** + * Contains the head of the linked array list if not null. The + * length is always capacityHint + 1 and the last element is an Object[] pointing + * to the next element of the linked array list. + */ + Object[] head; + /** The tail array where new elements will be added. */ + Object[] tail; + /** + * The total size of the list; written after elements have been added (release) and + * and when read, the value indicates how many elements can be safely read (acquire). + */ + volatile int size; + /** The next available slot in the current tail. */ + int indexInTail; + /** + * Constructor with the capacity hint of each array segment. + * @param capacityHint + */ + public LinkedArrayList(int capacityHint) { + this.capacityHint = capacityHint; + } + /** + * Adds a new element to this list. + * @param o the object to add, nulls are accepted + */ + public void add(Object o) { + // if no value yet, create the first array + if (size == 0) { + head = new Object[capacityHint + 1]; + tail = head; + head[0] = o; + indexInTail = 1; + size = 1; + } else + // if the tail is full, create a new tail and link + if (indexInTail == capacityHint) { + Object[] t = new Object[capacityHint + 1]; + t[0] = o; + tail[capacityHint] = t; + tail = t; + indexInTail = 1; + size++; + } else { + tail[indexInTail] = o; + indexInTail++; + size++; + } + } + /** + * Returns the head buffer segment or null if the list is empty. + * @return + */ + public Object[] head() { + return head; + } + /** + * Returns the tail buffer segment or null if the list is empty. + * @return + */ + public Object[] tail() { + return tail; + } + /** + * Returns the total size of the list. + * @return + */ + public int size() { + return size; + } + /** + * Returns the index of the next slot in the tail buffer segment. + * @return + */ + public int indexInTail() { + return indexInTail; + } + /** + * Returns the capacity hint that indicates the capacity of each buffer segment. + * @return + */ + public int capacityHint() { + return capacityHint; + } + /* Test support */List toList() { + final int cap = capacityHint; + final int s = size; + final List list = new ArrayList(s + 1); + + Object[] h = head(); + int j = 0; + int k = 0; + while (j < s) { + list.add(h[k]); + j++; + if (++k == cap) { + k = 0; + h = (Object[])h[cap]; + } + } + + return list; + } + @Override + public String toString() { + return toList().toString(); + } +} \ No newline at end of file diff --git a/src/main/java/rx/internal/util/LinkedArrayListTest.java b/src/main/java/rx/internal/util/LinkedArrayListTest.java new file mode 100644 index 0000000000..1b7d34fa0b --- /dev/null +++ b/src/main/java/rx/internal/util/LinkedArrayListTest.java @@ -0,0 +1,37 @@ +/** + * Copyright 2014 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.internal.util; + +import java.util.*; +import static org.junit.Assert.*; + +import org.junit.Test; + +public class LinkedArrayListTest { + @Test + public void testAdd() { + LinkedArrayList list = new LinkedArrayList(16); + + List expected = new ArrayList(32); + for (int i = 0; i < 32; i++) { + list.add(i); + expected.add(i); + } + + assertEquals(expected, list.toList()); + assertEquals(32, list.size()); + } +} \ No newline at end of file diff --git a/src/test/java/rx/internal/operators/CachedObservableTest.java b/src/test/java/rx/internal/operators/CachedObservableTest.java new file mode 100644 index 0000000000..ec88045dcb --- /dev/null +++ b/src/test/java/rx/internal/operators/CachedObservableTest.java @@ -0,0 +1,293 @@ +/** + * Copyright 2014 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.internal.operators; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +import java.util.*; +import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicInteger; + +import org.junit.*; + +import rx.*; +import rx.Observable.OnSubscribe; +import rx.Observable; +import rx.exceptions.TestException; +import rx.functions.*; +import rx.observers.TestSubscriber; +import rx.schedulers.Schedulers; + +public class CachedObservableTest { + @Test + public void testColdReplayNoBackpressure() { + CachedObservable source = CachedObservable.from(Observable.range(0, 1000)); + + assertFalse("Source is connected!", source.isConnected()); + + TestSubscriber ts = new TestSubscriber(); + + source.subscribe(ts); + + assertTrue("Source is not connected!", source.isConnected()); + assertFalse("Subscribers retained!", source.hasObservers()); + + ts.assertNoErrors(); + ts.assertTerminalEvent(); + List onNextEvents = ts.getOnNextEvents(); + assertEquals(1000, onNextEvents.size()); + + for (int i = 0; i < 1000; i++) { + assertEquals((Integer)i, onNextEvents.get(i)); + } + } + @Test + public void testColdReplayBackpressure() { + CachedObservable source = CachedObservable.from(Observable.range(0, 1000)); + + assertFalse("Source is connected!", source.isConnected()); + + TestSubscriber ts = new TestSubscriber(); + ts.requestMore(10); + + source.subscribe(ts); + + assertTrue("Source is not connected!", source.isConnected()); + assertTrue("Subscribers not retained!", source.hasObservers()); + + ts.assertNoErrors(); + assertTrue(ts.getOnCompletedEvents().isEmpty()); + List onNextEvents = ts.getOnNextEvents(); + assertEquals(10, onNextEvents.size()); + + for (int i = 0; i < 10; i++) { + assertEquals((Integer)i, onNextEvents.get(i)); + } + + ts.unsubscribe(); + assertFalse("Subscribers retained!", source.hasObservers()); + } + + @Test + public void testCache() throws InterruptedException { + final AtomicInteger counter = new AtomicInteger(); + Observable o = Observable.create(new Observable.OnSubscribe() { + + @Override + public void call(final Subscriber observer) { + new Thread(new Runnable() { + + @Override + public void run() { + counter.incrementAndGet(); + System.out.println("published observable being executed"); + observer.onNext("one"); + observer.onCompleted(); + } + }).start(); + } + }).cache(); + + // we then expect the following 2 subscriptions to get that same value + final CountDownLatch latch = new CountDownLatch(2); + + // subscribe once + o.subscribe(new Action1() { + + @Override + public void call(String v) { + assertEquals("one", v); + System.out.println("v: " + v); + latch.countDown(); + } + }); + + // subscribe again + o.subscribe(new Action1() { + + @Override + public void call(String v) { + assertEquals("one", v); + System.out.println("v: " + v); + latch.countDown(); + } + }); + + if (!latch.await(1000, TimeUnit.MILLISECONDS)) { + fail("subscriptions did not receive values"); + } + assertEquals(1, counter.get()); + } + + @Test + public void testUnsubscribeSource() { + Action0 unsubscribe = mock(Action0.class); + Observable o = Observable.just(1).doOnUnsubscribe(unsubscribe).cache(); + o.subscribe(); + o.subscribe(); + o.subscribe(); + verify(unsubscribe, times(1)).call(); + } + + @Test + public void testTake() { + TestSubscriber ts = new TestSubscriber(); + + CachedObservable cached = CachedObservable.from(Observable.range(1, 100)); + cached.take(10).subscribe(ts); + + ts.assertNoErrors(); + ts.assertTerminalEvent(); + ts.assertReceivedOnNext(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)); + ts.assertUnsubscribed(); + assertFalse(cached.hasObservers()); + } + + @Test + public void testAsync() { + Observable source = Observable.range(1, 10000); + for (int i = 0; i < 100; i++) { + TestSubscriber ts1 = new TestSubscriber(); + + CachedObservable cached = CachedObservable.from(source); + + cached.observeOn(Schedulers.computation()).subscribe(ts1); + + ts1.awaitTerminalEvent(2, TimeUnit.SECONDS); + ts1.assertNoErrors(); + ts1.assertTerminalEvent(); + assertEquals(10000, ts1.getOnNextEvents().size()); + + TestSubscriber ts2 = new TestSubscriber(); + cached.observeOn(Schedulers.computation()).subscribe(ts2); + + ts2.awaitTerminalEvent(2, TimeUnit.SECONDS); + ts2.assertNoErrors(); + ts2.assertTerminalEvent(); + assertEquals(10000, ts2.getOnNextEvents().size()); + } + } + @Test + public void testAsyncComeAndGo() { + Observable source = Observable.interval(1, 1, TimeUnit.MILLISECONDS) + .take(1000) + .subscribeOn(Schedulers.io()); + CachedObservable cached = CachedObservable.from(source); + + Observable output = cached.observeOn(Schedulers.computation()); + + List> list = new ArrayList>(100); + for (int i = 0; i < 100; i++) { + TestSubscriber ts = new TestSubscriber(); + list.add(ts); + output.skip(i * 10).take(10).subscribe(ts); + } + + List expected = new ArrayList(); + for (int i = 0; i < 10; i++) { + expected.add((long)(i - 10)); + } + int j = 0; + for (TestSubscriber ts : list) { + ts.awaitTerminalEvent(3, TimeUnit.SECONDS); + ts.assertNoErrors(); + ts.assertTerminalEvent(); + + for (int i = j * 10; i < j * 10 + 10; i++) { + expected.set(i - j * 10, (long)i); + } + + ts.assertReceivedOnNext(expected); + + j++; + } + } + + @Test + public void testNoMissingBackpressureException() { + final int m = 4 * 1000 * 1000; + Observable firehose = Observable.create(new OnSubscribe() { + @Override + public void call(Subscriber t) { + for (int i = 0; i < m; i++) { + t.onNext(i); + } + t.onCompleted(); + } + }); + + TestSubscriber ts = new TestSubscriber(); + firehose.cache().observeOn(Schedulers.computation()).takeLast(100).subscribe(ts); + + ts.awaitTerminalEvent(3, TimeUnit.SECONDS); + ts.assertNoErrors(); + ts.assertTerminalEvent(); + + assertEquals(100, ts.getOnNextEvents().size()); + } + + @Test + public void testValuesAndThenError() { + Observable source = Observable.range(1, 10) + .concatWith(Observable.error(new TestException())) + .cache(); + + + TestSubscriber ts = new TestSubscriber(); + source.subscribe(ts); + + ts.assertReceivedOnNext(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)); + Assert.assertTrue(ts.getOnCompletedEvents().isEmpty()); + Assert.assertEquals(1, ts.getOnErrorEvents().size()); + + TestSubscriber ts2 = new TestSubscriber(); + source.subscribe(ts2); + + ts2.assertReceivedOnNext(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)); + Assert.assertTrue(ts2.getOnCompletedEvents().isEmpty()); + Assert.assertEquals(1, ts2.getOnErrorEvents().size()); + } + + @Test + public void unsafeChildThrows() { + final AtomicInteger count = new AtomicInteger(); + + Observable source = Observable.range(1, 100) + .doOnNext(new Action1() { + @Override + public void call(Integer t) { + count.getAndIncrement(); + } + }) + .cache(); + + TestSubscriber ts = new TestSubscriber() { + @Override + public void onNext(Integer t) { + throw new TestException(); + } + }; + + source.unsafeSubscribe(ts); + + Assert.assertEquals(100, count.get()); + + ts.assertNoValues(); + ts.assertNotCompleted(); + ts.assertError(TestException.class); + } +} \ No newline at end of file diff --git a/src/test/java/rx/internal/operators/OnSubscribeCacheTest.java b/src/test/java/rx/internal/operators/OnSubscribeCacheTest.java deleted file mode 100644 index 0d74cd878b..0000000000 --- a/src/test/java/rx/internal/operators/OnSubscribeCacheTest.java +++ /dev/null @@ -1,164 +0,0 @@ -/** - * Copyright 2014 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.internal.operators; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; - -import java.util.Arrays; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; - -import org.junit.Test; - -import rx.Observable; -import rx.Subscriber; -import rx.functions.Action0; -import rx.functions.Action1; -import rx.functions.Func1; -import rx.functions.Func2; -import rx.observers.TestSubscriber; -import rx.schedulers.Schedulers; -import rx.subjects.AsyncSubject; -import rx.subjects.BehaviorSubject; -import rx.subjects.PublishSubject; -import rx.subjects.ReplaySubject; -import rx.subjects.Subject; - -public class OnSubscribeCacheTest { - - @Test - public void testCache() throws InterruptedException { - final AtomicInteger counter = new AtomicInteger(); - Observable o = Observable.create(new Observable.OnSubscribe() { - - @Override - public void call(final Subscriber observer) { - new Thread(new Runnable() { - - @Override - public void run() { - counter.incrementAndGet(); - System.out.println("published observable being executed"); - observer.onNext("one"); - observer.onCompleted(); - } - }).start(); - } - }).cache(); - - // we then expect the following 2 subscriptions to get that same value - final CountDownLatch latch = new CountDownLatch(2); - - // subscribe once - o.subscribe(new Action1() { - - @Override - public void call(String v) { - assertEquals("one", v); - System.out.println("v: " + v); - latch.countDown(); - } - }); - - // subscribe again - o.subscribe(new Action1() { - - @Override - public void call(String v) { - assertEquals("one", v); - System.out.println("v: " + v); - latch.countDown(); - } - }); - - if (!latch.await(1000, TimeUnit.MILLISECONDS)) { - fail("subscriptions did not receive values"); - } - assertEquals(1, counter.get()); - } - - private void testWithCustomSubjectAndRepeat(Subject subject, Integer... expected) { - Observable source0 = Observable.just(1, 2, 3) - .subscribeOn(Schedulers.io()) - .flatMap(new Func1>() { - @Override - public Observable call(final Integer i) { - return Observable.timer(i * 20, TimeUnit.MILLISECONDS).map(new Func1() { - @Override - public Integer call(Long t1) { - return i; - } - }); - } - }); - - Observable source1 = Observable.create(new OnSubscribeCache(source0, subject)); - - Observable source2 = source1 - .repeat(4) - .zipWith(Observable.interval(0, 10, TimeUnit.MILLISECONDS, Schedulers.newThread()), new Func2() { - @Override - public Integer call(Integer t1, Long t2) { - return t1; - } - - }); - TestSubscriber ts = new TestSubscriber(); - source2.subscribe(ts); - - ts.awaitTerminalEvent(); - ts.assertNoErrors(); - System.out.println(ts.getOnNextEvents()); - ts.assertReceivedOnNext(Arrays.asList(expected)); - } - - @Test(timeout = 10000) - public void testWithAsyncSubjectAndRepeat() { - testWithCustomSubjectAndRepeat(AsyncSubject. create(), 3, 3, 3, 3); - } - - @Test(timeout = 10000) - public void testWithBehaviorSubjectAndRepeat() { - // BehaviorSubject just completes when repeated - testWithCustomSubjectAndRepeat(BehaviorSubject.create(0), 0, 1, 2, 3); - } - - @Test(timeout = 10000) - public void testWithPublishSubjectAndRepeat() { - // PublishSubject just completes when repeated - testWithCustomSubjectAndRepeat(PublishSubject. create(), 1, 2, 3); - } - - @Test - public void testWithReplaySubjectAndRepeat() { - testWithCustomSubjectAndRepeat(ReplaySubject. create(), 1, 2, 3, 1, 2, 3, 1, 2, 3, 1, 2, 3); - } - - @Test - public void testUnsubscribeSource() { - Action0 unsubscribe = mock(Action0.class); - Observable o = Observable.just(1).doOnUnsubscribe(unsubscribe).cache(); - o.subscribe(); - o.subscribe(); - o.subscribe(); - verify(unsubscribe, times(1)).call(); - } -} From 9263fc2834acaca5d93029951accfafd052c4f8a Mon Sep 17 00:00:00 2001 From: akarnokd Date: Mon, 27 Jul 2015 08:42:25 +0200 Subject: [PATCH 20/81] Movet LinkedArrayListTest to the test section. --- src/{main => test}/java/rx/internal/util/LinkedArrayListTest.java | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename src/{main => test}/java/rx/internal/util/LinkedArrayListTest.java (100%) diff --git a/src/main/java/rx/internal/util/LinkedArrayListTest.java b/src/test/java/rx/internal/util/LinkedArrayListTest.java similarity index 100% rename from src/main/java/rx/internal/util/LinkedArrayListTest.java rename to src/test/java/rx/internal/util/LinkedArrayListTest.java From e0476edfb7a8a647b805951431268a10c809a3dd Mon Sep 17 00:00:00 2001 From: akarnokd Date: Mon, 27 Jul 2015 08:41:02 +0200 Subject: [PATCH 21/81] Operator replay() now supports backpressure (again) --- src/main/java/rx/Observable.java | 213 ++- .../OnSubscribeMulticastSelector.java | 77 -- .../rx/internal/operators/OperatorReplay.java | 1192 ++++++++++++++++- .../operators/OperatorReplayTest.java | 399 +++++- 4 files changed, 1620 insertions(+), 261 deletions(-) delete mode 100644 src/main/java/rx/internal/operators/OnSubscribeMulticastSelector.java diff --git a/src/main/java/rx/Observable.java b/src/main/java/rx/Observable.java index a6aabc082e..0aafecbf79 100644 --- a/src/main/java/rx/Observable.java +++ b/src/main/java/rx/Observable.java @@ -25,7 +25,6 @@ import rx.observers.SafeSubscriber; import rx.plugins.*; import rx.schedulers.*; -import rx.subjects.*; import rx.subscriptions.Subscriptions; /** @@ -5987,9 +5986,9 @@ public Void call(Notification notification) { * *
*
Backpressure Support:
- *
This operator does not support backpressure because multicasting means the stream is "hot" with - * multiple subscribers. Each child will need to manage backpressure independently using operators such - * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
+ *
This operator supports backpressure. Note that the upstream requests are determined by the child + * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will + * request 100 elements from the underlying Observable sequence.
*
Scheduler:
*
This version of {@code replay} does not operate by default on a particular {@link Scheduler}.
*
@@ -5999,14 +5998,7 @@ public Void call(Notification notification) { * @see ReactiveX operators documentation: Replay */ public final ConnectableObservable replay() { - return new OperatorMulticast(this, new Func0>() { - - @Override - public Subject call() { - return ReplaySubject. create(); - } - - }); + return OperatorReplay.create(this); } /** @@ -6016,9 +6008,9 @@ public final ConnectableObservable replay() { * *
*
Backpressure Support:
- *
This operator does not support backpressure because multicasting means the stream is "hot" with - * multiple subscribers. Each child will need to manage backpressure independently using operators such - * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
+ *
This operator supports backpressure. Note that the upstream requests are determined by the child + * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will + * request 100 elements from the underlying Observable sequence.
*
Scheduler:
*
This version of {@code replay} does not operate by default on a particular {@link Scheduler}.
*
@@ -6033,12 +6025,12 @@ public final ConnectableObservable replay() { * @see ReactiveX operators documentation: Replay */ public final Observable replay(Func1, ? extends Observable> selector) { - return create(new OnSubscribeMulticastSelector(this, new Func0>() { + return OperatorReplay.multicastSelector(new Func0>() { @Override - public final Subject call() { - return ReplaySubject.create(); + public ConnectableObservable call() { + return Observable.this.replay(); } - }, selector)); + }, selector); } /** @@ -6049,9 +6041,9 @@ public final Subject call() { * *
*
Backpressure Support:
- *
This operator does not support backpressure because multicasting means the stream is "hot" with - * multiple subscribers. Each child will need to manage backpressure independently using operators such - * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
+ *
This operator supports backpressure. Note that the upstream requests are determined by the child + * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will + * request 100 elements from the underlying Observable sequence.
*
Scheduler:
*
This version of {@code replay} does not operate by default on a particular {@link Scheduler}.
*
@@ -6069,12 +6061,12 @@ public final Subject call() { * @see ReactiveX operators documentation: Replay */ public final Observable replay(Func1, ? extends Observable> selector, final int bufferSize) { - return create(new OnSubscribeMulticastSelector(this, new Func0>() { + return OperatorReplay.multicastSelector(new Func0>() { @Override - public final Subject call() { - return ReplaySubject.createWithSize(bufferSize); + public ConnectableObservable call() { + return Observable.this.replay(bufferSize); } - }, selector)); + }, selector); } /** @@ -6085,9 +6077,9 @@ public final Subject call() { * *
*
Backpressure Support:
- *
This operator does not support backpressure because multicasting means the stream is "hot" with - * multiple subscribers. Each child will need to manage backpressure independently using operators such - * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
+ *
This operator supports backpressure. Note that the upstream requests are determined by the child + * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will + * request 100 elements from the underlying Observable sequence.
*
Scheduler:
*
This version of {@code replay} operates by default on the {@code computation} {@link Scheduler}.
*
@@ -6121,9 +6113,9 @@ public final Observable replay(Func1, ? extends Obs * *
*
Backpressure Support:
- *
This operator does not support backpressure because multicasting means the stream is "hot" with - * multiple subscribers. Each child will need to manage backpressure independently using operators such - * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
+ *
This operator supports backpressure. Note that the upstream requests are determined by the child + * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will + * request 100 elements from the underlying Observable sequence.
*
Scheduler:
*
you specify which {@link Scheduler} this operator will use
*
@@ -6153,12 +6145,12 @@ public final Observable replay(Func1, ? extends Obs if (bufferSize < 0) { throw new IllegalArgumentException("bufferSize < 0"); } - return create(new OnSubscribeMulticastSelector(this, new Func0>() { + return OperatorReplay.multicastSelector(new Func0>() { @Override - public final Subject call() { - return ReplaySubject.createWithTimeAndSize(time, unit, bufferSize, scheduler); + public ConnectableObservable call() { + return Observable.this.replay(bufferSize, time, unit, scheduler); } - }, selector)); + }, selector); } /** @@ -6169,9 +6161,9 @@ public final Subject call() { * *
*
Backpressure Support:
- *
This operator does not support backpressure because multicasting means the stream is "hot" with - * multiple subscribers. Each child will need to manage backpressure independently using operators such - * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
+ *
This operator supports backpressure. Note that the upstream requests are determined by the child + * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will + * request 100 elements from the underlying Observable sequence.
*
Scheduler:
*
you specify which {@link Scheduler} this operator will use
*
@@ -6190,13 +6182,18 @@ public final Subject call() { * replaying no more than {@code bufferSize} notifications * @see ReactiveX operators documentation: Replay */ - public final Observable replay(Func1, ? extends Observable> selector, final int bufferSize, final Scheduler scheduler) { - return create(new OnSubscribeMulticastSelector(this, new Func0>() { + public final Observable replay(final Func1, ? extends Observable> selector, final int bufferSize, final Scheduler scheduler) { + return OperatorReplay.multicastSelector(new Func0>() { @Override - public final Subject call() { - return OperatorReplay. createScheduledSubject(ReplaySubject.createWithSize(bufferSize), scheduler); + public ConnectableObservable call() { + return Observable.this.replay(bufferSize); } - }, selector)); + }, new Func1, Observable>() { + @Override + public Observable call(Observable t) { + return selector.call(t).observeOn(scheduler); + } + }); } /** @@ -6207,9 +6204,9 @@ public final Subject call() { * *
*
Backpressure Support:
- *
This operator does not support backpressure because multicasting means the stream is "hot" with - * multiple subscribers. Each child will need to manage backpressure independently using operators such - * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
+ *
This operator supports backpressure. Note that the upstream requests are determined by the child + * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will + * request 100 elements from the underlying Observable sequence.
*
Scheduler:
*
This version of {@code replay} operates by default on the {@code computation} {@link Scheduler}.
*
@@ -6240,9 +6237,9 @@ public final Observable replay(Func1, ? extends Obs * *
*
Backpressure Support:
- *
This operator does not support backpressure because multicasting means the stream is "hot" with - * multiple subscribers. Each child will need to manage backpressure independently using operators such - * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
+ *
This operator supports backpressure. Note that the upstream requests are determined by the child + * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will + * request 100 elements from the underlying Observable sequence.
*
Scheduler:
*
you specify which {@link Scheduler} this operator will use
*
@@ -6264,12 +6261,12 @@ public final Observable replay(Func1, ? extends Obs * @see ReactiveX operators documentation: Replay */ public final Observable replay(Func1, ? extends Observable> selector, final long time, final TimeUnit unit, final Scheduler scheduler) { - return create(new OnSubscribeMulticastSelector(this, new Func0>() { + return OperatorReplay.multicastSelector(new Func0>() { @Override - public final Subject call() { - return ReplaySubject.createWithTime(time, unit, scheduler); + public ConnectableObservable call() { + return Observable.this.replay(time, unit, scheduler); } - }, selector)); + }, selector); } /** @@ -6279,9 +6276,9 @@ public final Subject call() { * *
*
Backpressure Support:
- *
This operator does not support backpressure because multicasting means the stream is "hot" with - * multiple subscribers. Each child will need to manage backpressure independently using operators such - * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
+ *
This operator supports backpressure. Note that the upstream requests are determined by the child + * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will + * request 100 elements from the underlying Observable sequence.
*
Scheduler:
*
you specify which {@link Scheduler} this operator will use
*
@@ -6298,13 +6295,18 @@ public final Subject call() { * replaying all items * @see ReactiveX operators documentation: Replay */ - public final Observable replay(Func1, ? extends Observable> selector, final Scheduler scheduler) { - return create(new OnSubscribeMulticastSelector(this, new Func0>() { + public final Observable replay(final Func1, ? extends Observable> selector, final Scheduler scheduler) { + return OperatorReplay.multicastSelector(new Func0>() { @Override - public final Subject call() { - return OperatorReplay.createScheduledSubject(ReplaySubject. create(), scheduler); + public ConnectableObservable call() { + return Observable.this.replay(); } - }, selector)); + }, new Func1, Observable>() { + @Override + public Observable call(Observable t) { + return selector.call(t).observeOn(scheduler); + } + }); } /** @@ -6316,9 +6318,9 @@ public final Subject call() { * *
*
Backpressure Support:
- *
This operator does not support backpressure because multicasting means the stream is "hot" with - * multiple subscribers. Each child will need to manage backpressure independently using operators such - * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
+ *
This operator supports backpressure. Note that the upstream requests are determined by the child + * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will + * request 100 elements from the underlying Observable sequence.
*
Scheduler:
*
This version of {@code replay} does not operate by default on a particular {@link Scheduler}.
*
@@ -6330,14 +6332,7 @@ public final Subject call() { * @see ReactiveX operators documentation: Replay */ public final ConnectableObservable replay(final int bufferSize) { - return new OperatorMulticast(this, new Func0>() { - - @Override - public Subject call() { - return ReplaySubject.createWithSize(bufferSize); - } - - }); + return OperatorReplay.create(this, bufferSize); } /** @@ -6349,9 +6344,9 @@ public final ConnectableObservable replay(final int bufferSize) { * *
*
Backpressure Support:
- *
This operator does not support backpressure because multicasting means the stream is "hot" with - * multiple subscribers. Each child will need to manage backpressure independently using operators such - * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
+ *
This operator supports backpressure. Note that the upstream requests are determined by the child + * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will + * request 100 elements from the underlying Observable sequence.
*
Scheduler:
*
This version of {@code replay} operates by default on the {@code computation} {@link Scheduler}.
*
@@ -6380,9 +6375,9 @@ public final ConnectableObservable replay(int bufferSize, long time, TimeUnit * *
*
Backpressure Support:
- *
This operator does not support backpressure because multicasting means the stream is "hot" with - * multiple subscribers. Each child will need to manage backpressure independently using operators such - * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
+ *
This operator supports backpressure. Note that the upstream requests are determined by the child + * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will + * request 100 elements from the underlying Observable sequence.
*
Scheduler:
*
you specify which {@link Scheduler} this operator will use
*
@@ -6406,14 +6401,7 @@ public final ConnectableObservable replay(final int bufferSize, final long ti if (bufferSize < 0) { throw new IllegalArgumentException("bufferSize < 0"); } - return new OperatorMulticast(this, new Func0>() { - - @Override - public Subject call() { - return ReplaySubject.createWithTimeAndSize(time, unit, bufferSize, scheduler); - } - - }); + return OperatorReplay.create(this, time, unit, scheduler, bufferSize); } /** @@ -6425,9 +6413,9 @@ public final ConnectableObservable replay(final int bufferSize, final long ti * *
*
Backpressure Support:
- *
This operator does not support backpressure because multicasting means the stream is "hot" with - * multiple subscribers. Each child will need to manage backpressure independently using operators such - * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
+ *
This operator supports backpressure. Note that the upstream requests are determined by the child + * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will + * request 100 elements from the underlying Observable sequence.
*
Scheduler:
*
you specify which {@link Scheduler} this operator will use
*
@@ -6441,14 +6429,7 @@ public final ConnectableObservable replay(final int bufferSize, final long ti * @see ReactiveX operators documentation: Replay */ public final ConnectableObservable replay(final int bufferSize, final Scheduler scheduler) { - return new OperatorMulticast(this, new Func0>() { - - @Override - public Subject call() { - return OperatorReplay.createScheduledSubject(ReplaySubject.createWithSize(bufferSize), scheduler); - } - - }); + return OperatorReplay.observeOn(replay(bufferSize), scheduler); } /** @@ -6460,9 +6441,9 @@ public final ConnectableObservable replay(final int bufferSize, final Schedul * *
*
Backpressure Support:
- *
This operator does not support backpressure because multicasting means the stream is "hot" with - * multiple subscribers. Each child will need to manage backpressure independently using operators such - * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
+ *
This operator supports backpressure. Note that the upstream requests are determined by the child + * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will + * request 100 elements from the underlying Observable sequence.
*
Scheduler:
*
This version of {@code replay} operates by default on the {@code computation} {@link Scheduler}.
*
@@ -6488,9 +6469,9 @@ public final ConnectableObservable replay(long time, TimeUnit unit) { * *
*
Backpressure Support:
- *
This operator does not support backpressure because multicasting means the stream is "hot" with - * multiple subscribers. Each child will need to manage backpressure independently using operators such - * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
+ *
This operator supports backpressure. Note that the upstream requests are determined by the child + * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will + * request 100 elements from the underlying Observable sequence.
*
Scheduler:
*
you specify which {@link Scheduler} this operator will use
*
@@ -6506,14 +6487,7 @@ public final ConnectableObservable replay(long time, TimeUnit unit) { * @see ReactiveX operators documentation: Replay */ public final ConnectableObservable replay(final long time, final TimeUnit unit, final Scheduler scheduler) { - return new OperatorMulticast(this, new Func0>() { - - @Override - public Subject call() { - return ReplaySubject.createWithTime(time, unit, scheduler); - } - - }); + return OperatorReplay.create(this, time, unit, scheduler); } /** @@ -6525,9 +6499,9 @@ public final ConnectableObservable replay(final long time, final TimeUnit uni * *
*
Backpressure Support:
- *
This operator does not support backpressure because multicasting means the stream is "hot" with - * multiple subscribers. Each child will need to manage backpressure independently using operators such - * as {@link #onBackpressureDrop} and {@link #onBackpressureBuffer}.
+ *
This operator supports backpressure. Note that the upstream requests are determined by the child + * Subscriber which requests the largest amount: i.e., two child Subscribers with requests of 10 and 100 will + * request 100 elements from the underlying Observable sequence.
*
Scheduler:
*
you specify which {@link Scheduler} this operator will use
*
@@ -6540,14 +6514,7 @@ public final ConnectableObservable replay(final long time, final TimeUnit uni * @see ReactiveX operators documentation: Replay */ public final ConnectableObservable replay(final Scheduler scheduler) { - return new OperatorMulticast(this, new Func0>() { - - @Override - public Subject call() { - return OperatorReplay.createScheduledSubject(ReplaySubject. create(), scheduler); - } - - }); + return OperatorReplay.observeOn(replay(), scheduler); } /** diff --git a/src/main/java/rx/internal/operators/OnSubscribeMulticastSelector.java b/src/main/java/rx/internal/operators/OnSubscribeMulticastSelector.java deleted file mode 100644 index d1457ca6ec..0000000000 --- a/src/main/java/rx/internal/operators/OnSubscribeMulticastSelector.java +++ /dev/null @@ -1,77 +0,0 @@ -/** - * Copyright 2014 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.internal.operators; - -import rx.Observable; -import rx.Observable.OnSubscribe; -import rx.Subscriber; -import rx.Subscription; -import rx.functions.Action1; -import rx.functions.Func0; -import rx.functions.Func1; -import rx.observables.ConnectableObservable; -import rx.observers.SafeSubscriber; -import rx.subjects.Subject; - -/** - * Returns an observable sequence that contains the elements of a sequence - * produced by multicasting the source sequence within a selector function. - * - * @see MSDN: Observable.Multicast - * - * @param the input value type - * @param the intermediate type - * @param the result type - */ -public final class OnSubscribeMulticastSelector implements OnSubscribe { - final Observable source; - final Func0> subjectFactory; - final Func1, ? extends Observable> resultSelector; - - public OnSubscribeMulticastSelector(Observable source, - Func0> subjectFactory, - Func1, ? extends Observable> resultSelector) { - this.source = source; - this.subjectFactory = subjectFactory; - this.resultSelector = resultSelector; - } - - @Override - public void call(Subscriber child) { - Observable observable; - ConnectableObservable connectable; - try { - connectable = new OperatorMulticast(source, subjectFactory); - - observable = resultSelector.call(connectable); - } catch (Throwable t) { - child.onError(t); - return; - } - - final SafeSubscriber s = new SafeSubscriber(child); - - observable.unsafeSubscribe(s); - - connectable.connect(new Action1() { - @Override - public void call(Subscription t1) { - s.add(t1); - } - }); - } - -} diff --git a/src/main/java/rx/internal/operators/OperatorReplay.java b/src/main/java/rx/internal/operators/OperatorReplay.java index 83c76dfe39..e1bf7aa352 100644 --- a/src/main/java/rx/internal/operators/OperatorReplay.java +++ b/src/main/java/rx/internal/operators/OperatorReplay.java @@ -15,93 +15,1181 @@ */ package rx.internal.operators; +import java.util.*; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.*; +import rx.*; import rx.Observable; -import rx.Observable.OnSubscribe; -import rx.Scheduler; -import rx.Subscriber; -import rx.subjects.Subject; +import rx.exceptions.Exceptions; +import rx.exceptions.OnErrorThrowable; +import rx.functions.*; +import rx.observables.ConnectableObservable; +import rx.schedulers.Timestamped; +import rx.subscriptions.Subscriptions; -/** - * Replay with limited buffer and/or time constraints. - * - * - * @see MSDN: Observable.Replay overloads - */ -public final class OperatorReplay { - /** Utility class. */ - private OperatorReplay() { - throw new IllegalStateException("No instances!"); +public final class OperatorReplay extends ConnectableObservable { + /** The source observable. */ + final Observable source; + /** Holds the current subscriber that is, will be or just was subscribed to the source observable. */ + final AtomicReference> current; + /** A factory that creates the appropriate buffer for the ReplaySubscriber. */ + final Func0> bufferFactory; + + @SuppressWarnings("rawtypes") + static final Func0 DEFAULT_UNBOUNDED_FACTORY = new Func0() { + @Override + public Object call() { + return new UnboundedReplayBuffer(16); + } + }; + + /** + * Given a connectable observable factory, it multicasts over the generated + * ConnectableObservable via a selector function. + * @param connectableFactory + * @param selector + * @return + */ + public static Observable multicastSelector( + final Func0> connectableFactory, + final Func1, ? extends Observable> selector) { + return Observable.create(new OnSubscribe() { + @Override + public void call(final Subscriber child) { + ConnectableObservable co; + Observable observable; + try { + co = connectableFactory.call(); + observable = selector.call(co); + } catch (Throwable e) { + Exceptions.throwIfFatal(e); + child.onError(e); + return; + } + + observable.subscribe(child); + + co.connect(new Action1() { + @Override + public void call(Subscription t) { + child.add(t); + } + }); + } + }); + } + + /** + * Child Subscribers will observe the events of the ConnectableObservable on the + * specified scheduler. + * @param co + * @param scheduler + * @return + */ + public static ConnectableObservable observeOn(final ConnectableObservable co, final Scheduler scheduler) { + final Observable observable = co.observeOn(scheduler); + OnSubscribe onSubscribe = new OnSubscribe() { + @Override + public void call(final Subscriber child) { + // apply observeOn and prevent calling onStart() again + observable.unsafeSubscribe(new Subscriber(child) { + @Override + public void onNext(T t) { + child.onNext(t); + } + @Override + public void onError(Throwable e) { + child.onError(e); + } + @Override + public void onCompleted() { + child.onCompleted(); + } + }); + } + }; + return new ConnectableObservable(onSubscribe) { + @Override + public void connect(Action1 connection) { + co.connect(connection); + } + }; + } + + /** + * Creates a replaying ConnectableObservable with an unbounded buffer. + * @param source + * @return + */ + @SuppressWarnings("unchecked") + public static ConnectableObservable create(Observable source) { + return create(source, DEFAULT_UNBOUNDED_FACTORY); + } + + /** + * Creates a replaying ConnectableObservable with a size bound buffer. + * @param source + * @param bufferSize + * @return + */ + public static ConnectableObservable create(Observable source, + final int bufferSize) { + if (bufferSize == Integer.MAX_VALUE) { + return create(source); + } + return create(source, new Func0>() { + @Override + public ReplayBuffer call() { + return new SizeBoundReplayBuffer(bufferSize); + } + }); } /** - * Creates a subject whose client observers will observe events - * propagated through the given wrapped subject. - * @param the element type - * @param subject the subject to wrap - * @param scheduler the target scheduler - * @return the created subject + * Creates a replaying ConnectableObservable with a time bound buffer. + * @param source + * @param maxAge + * @param unit + * @param scheduler + * @return */ - public static Subject createScheduledSubject(Subject subject, Scheduler scheduler) { - final Observable observedOn = subject.observeOn(scheduler); - SubjectWrapper s = new SubjectWrapper(new OnSubscribe() { + public static ConnectableObservable create(Observable source, + long maxAge, TimeUnit unit, Scheduler scheduler) { + return create(source, maxAge, unit, scheduler, Integer.MAX_VALUE); + } + /** + * Creates a replaying ConnectableObservable with a size and time bound buffer. + * @param source + * @param maxAge + * @param unit + * @param scheduler + * @param bufferSize + * @return + */ + public static ConnectableObservable create(Observable source, + long maxAge, TimeUnit unit, final Scheduler scheduler, final int bufferSize) { + final long maxAgeInMillis = unit.toMillis(maxAge); + return create(source, new Func0>() { @Override - public void call(Subscriber o) { - subscriberOf(observedOn).call(o); + public ReplayBuffer call() { + return new SizeAndTimeBoundReplayBuffer(bufferSize, maxAgeInMillis, scheduler); } - - }, subject); - return s; + }); } /** - * Return an OnSubscribeFunc which delegates the subscription to the given observable. - * - * @param the value type - * @param target the target observable - * @return the function that delegates the subscription to the target + * Creates a OperatorReplay instance to replay values of the given source observable. + * @param source the source observable + * @param bufferFactory the factory to instantiate the appropriate buffer when the observable becomes active + * @return the connectable observable */ - public static OnSubscribe subscriberOf(final Observable target) { - return new OnSubscribe() { + static ConnectableObservable create(Observable source, + final Func0> bufferFactory) { + // the current connection to source needs to be shared between the operator and its onSubscribe call + final AtomicReference> curr = new AtomicReference>(); + OnSubscribe onSubscribe = new OnSubscribe() { @Override - public void call(Subscriber t1) { - target.unsafeSubscribe(t1); + public void call(Subscriber child) { + // concurrent connection/disconnection may change the state, + // we loop to be atomic while the child subscribes + for (;;) { + // get the current subscriber-to-source + ReplaySubscriber r = curr.get(); + // if there isn't one + if (r == null) { + // create a new subscriber to source + ReplaySubscriber u = new ReplaySubscriber(curr, bufferFactory.call()); + // perform extra initialization to avoid 'this' to escape during construction + u.init(); + // let's try setting it as the current subscriber-to-source + if (!curr.compareAndSet(r, u)) { + // didn't work, maybe someone else did it or the current subscriber + // to source has just finished + continue; + } + // we won, let's use it going onwards + r = u; + } + + // create the backpressure-managing producer for this child + InnerProducer inner = new InnerProducer(r, child); + // we try to add it to the array of producers + // if it fails, no worries because we will still have its buffer + // so it is going to replay it for us + r.add(inner); + // the producer has been registered with the current subscriber-to-source so + // at least it will receive the next terminal event + child.add(inner); + // setting the producer will trigger the first request to be considered by + // the subscriber-to-source. + child.setProducer(inner); + break; + } } }; + return new OperatorReplay(onSubscribe, source, curr, bufferFactory); + } + private OperatorReplay(OnSubscribe onSubscribe, Observable source, + final AtomicReference> current, + final Func0> bufferFactory) { + super(onSubscribe); + this.source = source; + this.current = current; + this.bufferFactory = bufferFactory; + } + + @Override + public void connect(Action1 connection) { + boolean doConnect = false; + ReplaySubscriber ps; + // we loop because concurrent connect/disconnect and termination may change the state + for (;;) { + // retrieve the current subscriber-to-source instance + ps = current.get(); + // if there is none yet or the current has unsubscribed + if (ps == null || ps.isUnsubscribed()) { + // create a new subscriber-to-source + ReplaySubscriber u = new ReplaySubscriber(current, bufferFactory.call()); + // initialize out the constructor to avoid 'this' to escape + u.init(); + // try setting it as the current subscriber-to-source + if (!current.compareAndSet(ps, u)) { + // did not work, perhaps a new subscriber arrived + // and created a new subscriber-to-source as well, retry + continue; + } + ps = u; + } + // if connect() was called concurrently, only one of them should actually + // connect to the source + doConnect = !ps.shouldConnect.get() && ps.shouldConnect.compareAndSet(false, true); + break; + } + /* + * Notify the callback that we have a (new) connection which it can unsubscribe + * but since ps is unique to a connection, multiple calls to connect() will return the + * same Subscription and even if there was a connect-disconnect-connect pair, the older + * references won't disconnect the newer connection. + * Synchronous source consumers have the opportunity to disconnect via unsubscribe on the + * Subscription as unsafeSubscribe may never return in its own. + * + * Note however, that asynchronously disconnecting a running source might leave + * child-subscribers without any terminal event; ReplaySubject does not have this + * issue because the unsubscription was always triggered by the child-subscribers + * themselves. + */ + connection.call(ps); + if (doConnect) { + source.unsafeSubscribe(ps); + } } + + @SuppressWarnings("rawtypes") + static final class ReplaySubscriber extends Subscriber implements Subscription { + /** Holds notifications from upstream. */ + final ReplayBuffer buffer; + /** The notification-lite factory. */ + final NotificationLite nl; + /** Contains either an onCompleted or an onError token from upstream. */ + boolean done; + + /** Indicates an empty array of inner producers. */ + static final InnerProducer[] EMPTY = new InnerProducer[0]; + /** Indicates a terminated ReplaySubscriber. */ + static final InnerProducer[] TERMINATED = new InnerProducer[0]; + + /** Tracks the subscribed producers. */ + final AtomicReference producers; + /** + * Atomically changed from false to true by connect to make sure the + * connection is only performed by one thread. + */ + final AtomicBoolean shouldConnect; + + /** Guarded by this. */ + boolean emitting; + /** Guarded by this. */ + boolean missed; + + + /** Contains the maximum element index the child Subscribers requested so far. Accessed while emitting is true. */ + long maxChildRequested; + /** Counts the outstanding upstream requests until the producer arrives. */ + long maxUpstreamRequested; + /** The upstream producer. */ + volatile Producer producer; + + public ReplaySubscriber(AtomicReference> current, + ReplayBuffer buffer) { + this.buffer = buffer; + + this.nl = NotificationLite.instance(); + this.producers = new AtomicReference(EMPTY); + this.shouldConnect = new AtomicBoolean(); + // make sure the source doesn't produce values until the child subscribers + // expressed their request amounts + this.request(0); + } + /** Should be called after the constructor finished to setup nulling-out the current reference. */ + void init() { + add(Subscriptions.create(new Action0() { + @Override + public void call() { + ReplaySubscriber.this.producers.getAndSet(TERMINATED); + // unlike OperatorPublish, we can't null out the terminated so + // late subscribers can still get replay + // current.compareAndSet(ReplaySubscriber.this, null); + // we don't care if it fails because it means the current has + // been replaced in the meantime + } + })); + } + /** + * Atomically try adding a new InnerProducer to this Subscriber or return false if this + * Subscriber was terminated. + * @param producer the producer to add + * @return true if succeeded, false otherwise + */ + boolean add(InnerProducer producer) { + if (producer == null) { + throw new NullPointerException(); + } + // the state can change so we do a CAS loop to achieve atomicity + for (;;) { + // get the current producer array + InnerProducer[] c = producers.get(); + // if this subscriber-to-source reached a terminal state by receiving + // an onError or onCompleted, just refuse to add the new producer + if (c == TERMINATED) { + return false; + } + // we perform a copy-on-write logic + int len = c.length; + InnerProducer[] u = new InnerProducer[len + 1]; + System.arraycopy(c, 0, u, 0, len); + u[len] = producer; + // try setting the producers array + if (producers.compareAndSet(c, u)) { + return true; + } + // if failed, some other operation succeded (another add, remove or termination) + // so retry + } + } + + /** + * Atomically removes the given producer from the producers array. + * @param producer the producer to remove + */ + void remove(InnerProducer producer) { + // the state can change so we do a CAS loop to achieve atomicity + for (;;) { + // let's read the current producers array + InnerProducer[] c = producers.get(); + // if it is either empty or terminated, there is nothing to remove so we quit + if (c == EMPTY || c == TERMINATED) { + return; + } + // let's find the supplied producer in the array + // although this is O(n), we don't expect too many child subscribers in general + int j = -1; + int len = c.length; + for (int i = 0; i < len; i++) { + if (c[i].equals(producer)) { + j = i; + break; + } + } + // we didn't find it so just quit + if (j < 0) { + return; + } + // we do copy-on-write logic here + InnerProducer[] u; + // we don't create a new empty array if producer was the single inhabitant + // but rather reuse an empty array + if (len == 1) { + u = EMPTY; + } else { + // otherwise, create a new array one less in size + u = new InnerProducer[len - 1]; + // copy elements being before the given producer + System.arraycopy(c, 0, u, 0, j); + // copy elements being after the given producer + System.arraycopy(c, j + 1, u, j, len - j - 1); + } + // try setting this new array as + if (producers.compareAndSet(c, u)) { + return; + } + // if we failed, it means something else happened + // (a concurrent add/remove or termination), we need to retry + } + } + + @Override + public void setProducer(Producer p) { + Producer p0 = producer; + if (p0 != null) { + throw new IllegalStateException("Only a single producer can be set on a Subscriber."); + } + producer = p; + manageRequests(); + replay(); + } + + @Override + public void onNext(T t) { + if (!done) { + buffer.next(t); + replay(); + } + } + @Override + public void onError(Throwable e) { + // The observer front is accessed serially as required by spec so + // no need to CAS in the terminal value + if (!done) { + done = true; + try { + buffer.error(e); + replay(); + } finally { + unsubscribe(); // expectation of testIssue2191 + } + } + } + @Override + public void onCompleted() { + // The observer front is accessed serially as required by spec so + // no need to CAS in the terminal value + if (!done) { + done = true; + try { + buffer.complete(); + replay(); + } finally { + unsubscribe(); + } + } + } + + /** + * Coordinates the request amounts of various child Subscribers. + */ + void manageRequests() { + // if the upstream has completed, no more requesting is possible + if (isUnsubscribed()) { + return; + } + synchronized (this) { + if (emitting) { + missed = true; + return; + } + emitting = true; + } + for (;;) { + // if the upstream has completed, no more requesting is possible + if (isUnsubscribed()) { + return; + } + + @SuppressWarnings("unchecked") + InnerProducer[] a = producers.get(); + + long ri = maxChildRequested; + long maxTotalRequests = 0; + + for (InnerProducer rp : a) { + maxTotalRequests = Math.max(maxTotalRequests, rp.totalRequested.get()); + } + + long ur = maxUpstreamRequested; + Producer p = producer; + long diff = maxTotalRequests - ri; + if (diff != 0) { + maxChildRequested = maxTotalRequests; + if (p != null) { + if (ur != 0L) { + maxUpstreamRequested = 0L; + p.request(ur + diff); + } else { + p.request(diff); + } + } else { + // collect upstream request amounts until there is a producer for them + long u = ur + diff; + if (u < 0) { + u = Long.MAX_VALUE; + } + maxUpstreamRequested = u; + } + } else + // if there were outstanding upstream requests and we have a producer + if (ur != 0L && p != null) { + maxUpstreamRequested = 0L; + // fire the accumulated requests + p.request(ur); + } + + synchronized (this) { + if (!missed) { + emitting = false; + return; + } + missed = false; + } + } + } + + /** + * Tries to replay the buffer contents to all known subscribers. + */ + void replay() { + @SuppressWarnings("unchecked") + InnerProducer[] a = producers.get(); + for (InnerProducer rp : a) { + buffer.replay(rp); + } + } + } /** - * A subject that wraps another subject. + * A Producer and Subscription that manages the request and unsubscription state of a + * child subscriber in thread-safe manner. + * We use AtomicLong as a base class to save on extra allocation of an AtomicLong and also + * save the overhead of the AtomicIntegerFieldUpdater. * @param the value type */ - public static final class SubjectWrapper extends Subject { - /** The wrapped subject. */ - final Subject subject; + static final class InnerProducer extends AtomicLong implements Producer, Subscription { + /** */ + private static final long serialVersionUID = -4453897557930727610L; + /** + * The parent subscriber-to-source used to allow removing the child in case of + * child unsubscription. + */ + final ReplaySubscriber parent; + /** The actual child subscriber. */ + final Subscriber child; + /** + * Holds an object that represents the current location in the buffer. + * Guarded by the emitter loop. + */ + Object index; + /** + * Keeps the sum of all requested amounts. + */ + final AtomicLong totalRequested; + /** Indicates an emission state. Guarded by this. */ + boolean emitting; + /** Indicates a missed update. Guarded by this. */ + boolean missed; + /** + * Indicates this child has been unsubscribed: the state is swapped in atomically and + * will prevent the dispatch() to emit (too many) values to a terminated child subscriber. + */ + static final long UNSUBSCRIBED = Long.MIN_VALUE; + + public InnerProducer(ReplaySubscriber parent, Subscriber child) { + this.parent = parent; + this.child = child; + this.totalRequested = new AtomicLong(); + } + + @Override + public void request(long n) { + // ignore negative requests + if (n < 0) { + return; + } + // In general, RxJava doesn't prevent concurrent requests (with each other or with + // an unsubscribe) so we need a CAS-loop, but we need to handle + // request overflow and unsubscribed/not requested state as well. + for (;;) { + // get the current request amount + long r = get(); + // if child called unsubscribe() do nothing + if (r == UNSUBSCRIBED) { + return; + } + // ignore zero requests except any first that sets in zero + if (r >= 0L && n == 0) { + return; + } + // otherwise, increase the request count + long u = r + n; + // and check for long overflow + if (u < 0) { + // cap at max value, which is essentially unlimited + u = Long.MAX_VALUE; + } + // try setting the new request value + if (compareAndSet(r, u)) { + // increment the total request counter + addTotalRequested(n); + // if successful, notify the parent dispacher this child can receive more + // elements + parent.manageRequests(); + + parent.buffer.replay(this); + return; + } + // otherwise, someone else changed the state (perhaps a concurrent + // request or unsubscription so retry + } + } + + /** + * Increments the total requested amount. + * @param n the additional request amount + */ + void addTotalRequested(long n) { + for (;;) { + long r = totalRequested.get(); + long u = r + n; + if (u < 0) { + u = Long.MAX_VALUE; + } + if (totalRequested.compareAndSet(r, u)) { + return; + } + } + } + + /** + * Indicate that values have been emitted to this child subscriber by the dispatch() method. + * @param n the number of items emitted + * @return the updated request value (may indicate how much can be produced or a terminal state) + */ + public long produced(long n) { + // we don't allow producing zero or less: it would be a bug in the operator + if (n <= 0) { + throw new IllegalArgumentException("Cant produce zero or less"); + } + for (;;) { + // get the current request value + long r = get(); + // if the child has unsubscribed, simply return and indicate this + if (r == UNSUBSCRIBED) { + return UNSUBSCRIBED; + } + // reduce the requested amount + long u = r - n; + // if the new amount is less than zero, we have a bug in this operator + if (u < 0) { + throw new IllegalStateException("More produced (" + n + ") than requested (" + r + ")"); + } + // try updating the request value + if (compareAndSet(r, u)) { + // and return the udpated value + return u; + } + // otherwise, some concurrent activity happened and we need to retry + } + } + + @Override + public boolean isUnsubscribed() { + return get() == UNSUBSCRIBED; + } + @Override + public void unsubscribe() { + long r = get(); + // let's see if we are unsubscribed + if (r != UNSUBSCRIBED) { + // if not, swap in the terminal state, this is idempotent + // because other methods using CAS won't overwrite this value, + // concurrent calls to unsubscribe will atomically swap in the same + // terminal value + r = getAndSet(UNSUBSCRIBED); + // and only one of them will see a non-terminated value before the swap + if (r != UNSUBSCRIBED) { + // remove this from the parent + parent.remove(this); + // After removal, we might have unblocked the other child subscribers: + // let's assume this child had 0 requested before the unsubscription while + // the others had non-zero. By removing this 'blocking' child, the others + // are now free to receive events + parent.manageRequests(); + } + } + } + /** + * Convenience method to auto-cast the index object. + * @return + */ + @SuppressWarnings("unchecked") + U index() { + return (U)index; + } + } + /** + * The interface for interacting with various buffering logic. + * + * @param the value type + */ + interface ReplayBuffer { + /** + * Adds a regular value to the buffer. + * @param value + */ + void next(T value); + /** + * Adds a terminal exception to the buffer + * @param e + */ + void error(Throwable e); + /** + * Adds a completion event to the buffer + */ + void complete(); + /** + * Tries to replay the buffered values to the + * subscriber inside the output if there + * is new value and requests available at the + * same time. + * @param output + */ + void replay(InnerProducer output); + } + + /** + * Holds an unbounded list of events. + * + * @param the value type + */ + static final class UnboundedReplayBuffer extends ArrayList implements ReplayBuffer { + /** */ + private static final long serialVersionUID = 7063189396499112664L; + final NotificationLite nl; + /** The total number of events in the buffer. */ + volatile int size; + + public UnboundedReplayBuffer(int capacityHint) { + super(capacityHint); + nl = NotificationLite.instance(); + } + @Override + public void next(T value) { + add(nl.next(value)); + size++; + } - public SubjectWrapper(OnSubscribe func, Subject subject) { - super(func); - this.subject = subject; + @Override + public void error(Throwable e) { + add(nl.error(e)); + size++; } @Override - public void onNext(T args) { - subject.onNext(args); + public void complete() { + add(nl.completed()); + size++; } @Override - public void onError(Throwable e) { - subject.onError(e); + public void replay(InnerProducer output) { + synchronized (output) { + if (output.emitting) { + output.missed = true; + return; + } + output.emitting = true; + } + for (;;) { + if (output.isUnsubscribed()) { + return; + } + int sourceIndex = size; + + Integer destIndexObject = output.index(); + int destIndex = destIndexObject != null ? destIndexObject.intValue() : 0; + + long r = output.get(); + long r0 = r; + long e = 0L; + + while (r != 0L && destIndex < sourceIndex) { + Object o = get(destIndex); + try { + if (nl.accept(output.child, o)) { + return; + } + } catch (Throwable err) { + Exceptions.throwIfFatal(err); + output.unsubscribe(); + if (!nl.isError(o) && !nl.isCompleted(o)) { + output.child.onError(OnErrorThrowable.addValueAsLastCause(err, nl.getValue(o))); + } + return; + } + if (output.isUnsubscribed()) { + return; + } + destIndex++; + r--; + e++; + } + if (e != 0L) { + output.index = destIndex; + if (r0 != Long.MAX_VALUE) { + output.produced(e); + } + } + + synchronized (output) { + if (!output.missed) { + output.emitting = false; + return; + } + output.missed = false; + } + } + } + } + + /** + * Represents a node in a bounded replay buffer's linked list. + * + * @param the contained value type + */ + static final class Node extends AtomicReference { + /** */ + private static final long serialVersionUID = 245354315435971818L; + final Object value; + public Node(Object value) { + this.value = value; + } + } + + /** + * Base class for bounded buffering with options to specify an + * enter and leave transforms and custom truncation behavior. + * + * @param the value type + */ + static class BoundedReplayBuffer extends AtomicReference implements ReplayBuffer { + /** */ + private static final long serialVersionUID = 2346567790059478686L; + final NotificationLite nl; + + Node tail; + int size; + + public BoundedReplayBuffer() { + nl = NotificationLite.instance(); + Node n = new Node(null); + tail = n; + set(n); + } + + /** + * Add a new node to the linked list. + * @param n + */ + final void addLast(Node n) { + tail.set(n); + tail = n; + size++; + } + /** + * Remove the first node from the linked list. + */ + final void removeFirst() { + Node head = get(); + Node next = head.get(); + if (next == null) { + throw new IllegalStateException("Empty list!"); + } + size--; + // can't just move the head because it would retain the very first value + // can't null out the head's value because of late replayers would see null + setFirst(next); + } + /* test */ final void removeSome(int n) { + Node head = get(); + while (n > 0) { + head = head.get(); + n--; + size--; + } + + setFirst(head); + } + /** + * Arranges the given node is the new head from now on. + * @param n + */ + final void setFirst(Node n) { + set(n); + } + + @Override + public final void next(T value) { + Object o = enterTransform(nl.next(value)); + Node n = new Node(o); + addLast(n); + truncate(); } @Override - public void onCompleted() { - subject.onCompleted(); + public final void error(Throwable e) { + Object o = enterTransform(nl.error(e)); + Node n = new Node(o); + addLast(n); + truncateFinal(); } @Override - public boolean hasObservers() { - return this.subject.hasObservers(); + public final void complete() { + Object o = enterTransform(nl.completed()); + Node n = new Node(o); + addLast(n); + truncateFinal(); + } + + @Override + public final void replay(InnerProducer output) { + synchronized (output) { + if (output.emitting) { + output.missed = true; + return; + } + output.emitting = true; + } + for (;;) { + if (output.isUnsubscribed()) { + return; + } + + long r = output.get(); + long r0 = r; + long e = 0L; + + Node node = output.index(); + if (node == null) { + node = get(); + output.index = node; + } + + while (r != 0) { + Node v = node.get(); + if (v != null) { + Object o = leaveTransform(v.value); + try { + if (nl.accept(output.child, o)) { + output.index = null; + return; + } + } catch (Throwable err) { + output.index = null; + Exceptions.throwIfFatal(err); + output.unsubscribe(); + if (!nl.isError(o) && !nl.isCompleted(o)) { + output.child.onError(OnErrorThrowable.addValueAsLastCause(err, nl.getValue(o))); + } + return; + } + e++; + node = v; + } else { + break; + } + if (output.isUnsubscribed()) { + return; + } + } + + if (e != 0L) { + output.index = node; + if (r0 != Long.MAX_VALUE) { + output.produced(e); + } + } + + synchronized (output) { + if (!output.missed) { + output.emitting = false; + return; + } + output.missed = false; + } + } + + } + + /** + * Override this to wrap the NotificationLite object into a + * container to be used later by truncate. + * @param value + * @return + */ + Object enterTransform(Object value) { + return value; + } + /** + * Override this to unwrap the transformed value into a + * NotificationLite object. + * @param value + * @return + */ + Object leaveTransform(Object value) { + return value; + } + /** + * Override this method to truncate a non-terminated buffer + * based on its current properties. + */ + void truncate() { + + } + /** + * Override this method to truncate a terminated buffer + * based on its properties (i.e., truncate but the very last node). + */ + void truncateFinal() { + + } + /* test */ final void collect(Collection output) { + Node n = get(); + for (;;) { + Node next = n.get(); + if (next != null) { + Object o = next.value; + Object v = leaveTransform(o); + if (nl.isCompleted(v) || nl.isError(v)) { + break; + } + output.add(nl.getValue(v)); + n = next; + } else { + break; + } + } + } + /* test */ boolean hasError() { + return tail.value != null && nl.isError(leaveTransform(tail.value)); + } + /* test */ boolean hasCompleted() { + return tail.value != null && nl.isCompleted(leaveTransform(tail.value)); + } + } + + /** + * A bounded replay buffer implementation with size limit only. + * + * @param the value type + */ + static final class SizeBoundReplayBuffer extends BoundedReplayBuffer { + /** */ + private static final long serialVersionUID = -5898283885385201806L; + + final int limit; + public SizeBoundReplayBuffer(int limit) { + this.limit = limit; + } + + @Override + void truncate() { + // overflow can be at most one element + if (size > limit) { + removeFirst(); + } + } + + // no need for final truncation because values are truncated one by one + } + + /** + * Size and time bound replay buffer. + * + * @param the buffered value type + */ + static final class SizeAndTimeBoundReplayBuffer extends BoundedReplayBuffer { + /** */ + private static final long serialVersionUID = 3457957419649567404L; + final Scheduler scheduler; + final long maxAgeInMillis; + final int limit; + public SizeAndTimeBoundReplayBuffer(int limit, long maxAgeInMillis, Scheduler scheduler) { + this.scheduler = scheduler; + this.limit = limit; + this.maxAgeInMillis = maxAgeInMillis; + } + + @Override + Object enterTransform(Object value) { + return new Timestamped(scheduler.now(), value); + } + + @Override + Object leaveTransform(Object value) { + return ((Timestamped)value).getValue(); + } + + @Override + void truncate() { + long timeLimit = scheduler.now() - maxAgeInMillis; + + Node prev = get(); + Node next = prev.get(); + + int e = 0; + for (;;) { + if (next != null) { + if (size > limit) { + e++; + size--; + prev = next; + next = next.get(); + } else { + Timestamped v = (Timestamped)next.value; + if (v.getTimestampMillis() <= timeLimit) { + e++; + size--; + prev = next; + next = next.get(); + } else { + break; + } + } + } else { + break; + } + } + if (e != 0) { + setFirst(prev); + } + } + @Override + void truncateFinal() { + long timeLimit = scheduler.now() - maxAgeInMillis; + + Node prev = get(); + Node next = prev.get(); + + int e = 0; + for (;;) { + if (next != null && size > 1) { + Timestamped v = (Timestamped)next.value; + if (v.getTimestampMillis() <= timeLimit) { + e++; + size--; + prev = next; + next = next.get(); + } else { + break; + } + } else { + break; + } + } + if (e != 0) { + setFirst(prev); + } } } } \ No newline at end of file diff --git a/src/test/java/rx/internal/operators/OperatorReplayTest.java b/src/test/java/rx/internal/operators/OperatorReplayTest.java index a5ff85864d..046803b082 100644 --- a/src/test/java/rx/internal/operators/OperatorReplayTest.java +++ b/src/test/java/rx/internal/operators/OperatorReplayTest.java @@ -15,33 +15,40 @@ */ package rx.internal.operators; -import static org.junit.Assert.assertEquals; +import static org.junit.Assert.*; import static org.mockito.Matchers.any; import static org.mockito.Matchers.notNull; -import static org.mockito.Mockito.inOrder; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyNoMoreInteractions; -import static org.mockito.Mockito.when; +import static org.mockito.Mockito.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import org.junit.Assert; import org.junit.Test; import org.mockito.InOrder; import rx.Observable; +import rx.Observable.OnSubscribe; import rx.Observer; import rx.Scheduler; import rx.Scheduler.Worker; +import rx.Subscriber; import rx.Subscription; +import rx.exceptions.TestException; import rx.functions.Action0; import rx.functions.Action1; import rx.functions.Func1; +import rx.internal.operators.OperatorReplay.BoundedReplayBuffer; +import rx.internal.operators.OperatorReplay.Node; +import rx.internal.operators.OperatorReplay.SizeAndTimeBoundReplayBuffer; import rx.observables.ConnectableObservable; +import rx.observers.TestSubscriber; +import rx.schedulers.Schedulers; import rx.schedulers.TestScheduler; import rx.subjects.PublishSubject; @@ -739,4 +746,378 @@ public boolean isUnsubscribed() { } } + @Test + public void testBoundedReplayBuffer() { + BoundedReplayBuffer buf = new BoundedReplayBuffer(); + buf.addLast(new Node(1)); + buf.addLast(new Node(2)); + buf.addLast(new Node(3)); + buf.addLast(new Node(4)); + buf.addLast(new Node(5)); + + List values = new ArrayList(); + buf.collect(values); + + Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5), values); + + buf.removeSome(2); + buf.removeFirst(); + buf.removeSome(2); + + values.clear(); + buf.collect(values); + Assert.assertTrue(values.isEmpty()); + + buf.addLast(new Node(5)); + buf.addLast(new Node(6)); + buf.collect(values); + + Assert.assertEquals(Arrays.asList(5, 6), values); + + } + + @Test + public void testTimedAndSizedTruncation() { + TestScheduler test = Schedulers.test(); + SizeAndTimeBoundReplayBuffer buf = new SizeAndTimeBoundReplayBuffer(2, 2000, test); + List values = new ArrayList(); + + buf.next(1); + test.advanceTimeBy(1, TimeUnit.SECONDS); + buf.next(2); + test.advanceTimeBy(1, TimeUnit.SECONDS); + buf.collect(values); + Assert.assertEquals(Arrays.asList(1, 2), values); + + buf.next(3); + buf.next(4); + values.clear(); + buf.collect(values); + Assert.assertEquals(Arrays.asList(3, 4), values); + + test.advanceTimeBy(2, TimeUnit.SECONDS); + buf.next(5); + + values.clear(); + buf.collect(values); + Assert.assertEquals(Arrays.asList(5), values); + + test.advanceTimeBy(2, TimeUnit.SECONDS); + buf.complete(); + + values.clear(); + buf.collect(values); + Assert.assertTrue(values.isEmpty()); + + Assert.assertEquals(1, buf.size); + Assert.assertTrue(buf.hasCompleted()); + } + + @Test + public void testBackpressure() { + final AtomicLong requested = new AtomicLong(); + Observable source = Observable.range(1, 1000) + .doOnRequest(new Action1() { + @Override + public void call(Long t) { + requested.addAndGet(t); + } + }); + ConnectableObservable co = source.replay(); + + TestSubscriber ts1 = TestSubscriber.create(10); + TestSubscriber ts2 = TestSubscriber.create(90); + + co.subscribe(ts1); + co.subscribe(ts2); + + ts2.requestMore(10); + + co.connect(); + + ts1.assertValueCount(10); + ts1.assertNoTerminalEvent(); + + ts2.assertValueCount(100); + ts2.assertNoTerminalEvent(); + + Assert.assertEquals(100, requested.get()); + } + + @Test + public void testBackpressureBounded() { + final AtomicLong requested = new AtomicLong(); + Observable source = Observable.range(1, 1000) + .doOnRequest(new Action1() { + @Override + public void call(Long t) { + requested.addAndGet(t); + } + }); + ConnectableObservable co = source.replay(50); + + TestSubscriber ts1 = TestSubscriber.create(10); + TestSubscriber ts2 = TestSubscriber.create(90); + + co.subscribe(ts1); + co.subscribe(ts2); + + ts2.requestMore(10); + + co.connect(); + + ts1.assertValueCount(10); + ts1.assertNoTerminalEvent(); + + ts2.assertValueCount(100); + ts2.assertNoTerminalEvent(); + + Assert.assertEquals(100, requested.get()); + } + + @Test + public void testColdReplayNoBackpressure() { + Observable source = Observable.range(0, 1000).replay().autoConnect(); + + TestSubscriber ts = new TestSubscriber(); + + source.subscribe(ts); + + ts.assertNoErrors(); + ts.assertTerminalEvent(); + List onNextEvents = ts.getOnNextEvents(); + assertEquals(1000, onNextEvents.size()); + + for (int i = 0; i < 1000; i++) { + assertEquals((Integer)i, onNextEvents.get(i)); + } + } + @Test + public void testColdReplayBackpressure() { + Observable source = Observable.range(0, 1000).replay().autoConnect(); + + TestSubscriber ts = new TestSubscriber(); + ts.requestMore(10); + + source.subscribe(ts); + + ts.assertNoErrors(); + assertTrue(ts.getOnCompletedEvents().isEmpty()); + List onNextEvents = ts.getOnNextEvents(); + assertEquals(10, onNextEvents.size()); + + for (int i = 0; i < 10; i++) { + assertEquals((Integer)i, onNextEvents.get(i)); + } + + ts.unsubscribe(); + } + + @Test + public void testCache() throws InterruptedException { + final AtomicInteger counter = new AtomicInteger(); + Observable o = Observable.create(new Observable.OnSubscribe() { + + @Override + public void call(final Subscriber observer) { + new Thread(new Runnable() { + + @Override + public void run() { + counter.incrementAndGet(); + System.out.println("published observable being executed"); + observer.onNext("one"); + observer.onCompleted(); + } + }).start(); + } + }).replay().autoConnect(); + + // we then expect the following 2 subscriptions to get that same value + final CountDownLatch latch = new CountDownLatch(2); + + // subscribe once + o.subscribe(new Action1() { + + @Override + public void call(String v) { + assertEquals("one", v); + System.out.println("v: " + v); + latch.countDown(); + } + }); + + // subscribe again + o.subscribe(new Action1() { + + @Override + public void call(String v) { + assertEquals("one", v); + System.out.println("v: " + v); + latch.countDown(); + } + }); + + if (!latch.await(1000, TimeUnit.MILLISECONDS)) { + fail("subscriptions did not receive values"); + } + assertEquals(1, counter.get()); + } + + @Test + public void testUnsubscribeSource() { + Action0 unsubscribe = mock(Action0.class); + Observable o = Observable.just(1).doOnUnsubscribe(unsubscribe).cache(); + o.subscribe(); + o.subscribe(); + o.subscribe(); + verify(unsubscribe, times(1)).call(); + } + + @Test + public void testTake() { + TestSubscriber ts = new TestSubscriber(); + + Observable cached = Observable.range(1, 100).replay().autoConnect(); + cached.take(10).subscribe(ts); + + ts.assertNoErrors(); + ts.assertTerminalEvent(); + ts.assertReceivedOnNext(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)); + ts.assertUnsubscribed(); + } + + @Test + public void testAsync() { + Observable source = Observable.range(1, 10000); + for (int i = 0; i < 100; i++) { + TestSubscriber ts1 = new TestSubscriber(); + + Observable cached = source.replay().autoConnect(); + + cached.observeOn(Schedulers.computation()).subscribe(ts1); + + ts1.awaitTerminalEvent(2, TimeUnit.SECONDS); + ts1.assertNoErrors(); + ts1.assertTerminalEvent(); + assertEquals(10000, ts1.getOnNextEvents().size()); + + TestSubscriber ts2 = new TestSubscriber(); + cached.observeOn(Schedulers.computation()).subscribe(ts2); + + ts2.awaitTerminalEvent(2, TimeUnit.SECONDS); + ts2.assertNoErrors(); + ts2.assertTerminalEvent(); + assertEquals(10000, ts2.getOnNextEvents().size()); + } + } + @Test + public void testAsyncComeAndGo() { + Observable source = Observable.interval(1, 1, TimeUnit.MILLISECONDS) + .take(1000) + .subscribeOn(Schedulers.io()); + Observable cached = source.replay().autoConnect(); + + Observable output = cached.observeOn(Schedulers.computation()); + + List> list = new ArrayList>(100); + for (int i = 0; i < 100; i++) { + TestSubscriber ts = new TestSubscriber(); + list.add(ts); + output.skip(i * 10).take(10).subscribe(ts); + } + + List expected = new ArrayList(); + for (int i = 0; i < 10; i++) { + expected.add((long)(i - 10)); + } + int j = 0; + for (TestSubscriber ts : list) { + ts.awaitTerminalEvent(3, TimeUnit.SECONDS); + ts.assertNoErrors(); + ts.assertTerminalEvent(); + + for (int i = j * 10; i < j * 10 + 10; i++) { + expected.set(i - j * 10, (long)i); + } + + ts.assertReceivedOnNext(expected); + + j++; + } + } + + @Test + public void testNoMissingBackpressureException() { + final int m = 4 * 1000 * 1000; + Observable firehose = Observable.create(new OnSubscribe() { + @Override + public void call(Subscriber t) { + for (int i = 0; i < m; i++) { + t.onNext(i); + } + t.onCompleted(); + } + }); + + TestSubscriber ts = new TestSubscriber(); + firehose.replay().autoConnect().observeOn(Schedulers.computation()).takeLast(100).subscribe(ts); + + ts.awaitTerminalEvent(3, TimeUnit.SECONDS); + ts.assertNoErrors(); + ts.assertTerminalEvent(); + + assertEquals(100, ts.getOnNextEvents().size()); + } + + @Test + public void testValuesAndThenError() { + Observable source = Observable.range(1, 10) + .concatWith(Observable.error(new TestException())) + .replay().autoConnect(); + + + TestSubscriber ts = new TestSubscriber(); + source.subscribe(ts); + + ts.assertReceivedOnNext(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)); + Assert.assertTrue(ts.getOnCompletedEvents().isEmpty()); + Assert.assertEquals(1, ts.getOnErrorEvents().size()); + + TestSubscriber ts2 = new TestSubscriber(); + source.subscribe(ts2); + + ts2.assertReceivedOnNext(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)); + Assert.assertTrue(ts2.getOnCompletedEvents().isEmpty()); + Assert.assertEquals(1, ts2.getOnErrorEvents().size()); + } + + @Test + public void unsafeChildThrows() { + final AtomicInteger count = new AtomicInteger(); + + Observable source = Observable.range(1, 100) + .doOnNext(new Action1() { + @Override + public void call(Integer t) { + count.getAndIncrement(); + } + }) + .replay().autoConnect(); + + TestSubscriber ts = new TestSubscriber() { + @Override + public void onNext(Integer t) { + throw new TestException(); + } + }; + + source.unsafeSubscribe(ts); + + Assert.assertEquals(100, count.get()); + + ts.assertNoValues(); + ts.assertNotCompleted(); + ts.assertError(TestException.class); + } } \ No newline at end of file From 038e1acbe8a8863d2c90395948a56529b720b8db Mon Sep 17 00:00:00 2001 From: Yuya Tanaka Date: Thu, 30 Jul 2015 00:26:44 +0900 Subject: [PATCH 22/81] No InterruptedException with synchronous BlockingObservable --- .../operators/BlockingOperatorToIterator.java | 4 + .../rx/observables/BlockingObservable.java | 40 +-- .../operators/BlockingOperatorNextTest.java | 24 +- .../observables/BlockingObservableTest.java | 251 ++++++++++++------ 4 files changed, 215 insertions(+), 104 deletions(-) diff --git a/src/main/java/rx/internal/operators/BlockingOperatorToIterator.java b/src/main/java/rx/internal/operators/BlockingOperatorToIterator.java index da63d9e227..6f631a211d 100644 --- a/src/main/java/rx/internal/operators/BlockingOperatorToIterator.java +++ b/src/main/java/rx/internal/operators/BlockingOperatorToIterator.java @@ -93,6 +93,10 @@ public T next() { private Notification take() { try { + Notification poll = notifications.poll(); + if (poll != null) { + return poll; + } return notifications.take(); } catch (InterruptedException e) { subscription.unsubscribe(); diff --git a/src/main/java/rx/observables/BlockingObservable.java b/src/main/java/rx/observables/BlockingObservable.java index 8a4ce728cf..4f4fdda412 100644 --- a/src/main/java/rx/observables/BlockingObservable.java +++ b/src/main/java/rx/observables/BlockingObservable.java @@ -123,17 +123,7 @@ public void onNext(T args) { onNext.call(args); } }); - // block until the subscription completes and then return - try { - latch.await(); - } catch (InterruptedException e) { - subscription.unsubscribe(); - // set the interrupted flag again so callers can still get it - // for more information see https://github.com/ReactiveX/RxJava/pull/147#issuecomment-13624780 - Thread.currentThread().interrupt(); - // using Runtime so it is not checked - throw new RuntimeException("Interrupted while waiting for subscription to complete.", e); - } + awaitForComplete(latch, subscription); if (exceptionFromOnError.get() != null) { if (exceptionFromOnError.get() instanceof RuntimeException) { @@ -456,14 +446,7 @@ public void onNext(final T item) { returnItem.set(item); } }); - - try { - latch.await(); - } catch (InterruptedException e) { - subscription.unsubscribe(); - Thread.currentThread().interrupt(); - throw new RuntimeException("Interrupted while waiting for subscription to complete.", e); - } + awaitForComplete(latch, subscription); if (returnException.get() != null) { if (returnException.get() instanceof RuntimeException) { @@ -475,4 +458,23 @@ public void onNext(final T item) { return returnItem.get(); } + + private void awaitForComplete(CountDownLatch latch, Subscription subscription) { + if (latch.getCount() == 0) { + // Synchronous observable completes before awaiting for it. + // Skip await so InterruptedException will never be thrown. + return; + } + // block until the subscription completes and then return + try { + latch.await(); + } catch (InterruptedException e) { + subscription.unsubscribe(); + // set the interrupted flag again so callers can still get it + // for more information see https://github.com/ReactiveX/RxJava/pull/147#issuecomment-13624780 + Thread.currentThread().interrupt(); + // using Runtime so it is not checked + throw new RuntimeException("Interrupted while waiting for subscription to complete.", e); + } + } } diff --git a/src/test/java/rx/internal/operators/BlockingOperatorNextTest.java b/src/test/java/rx/internal/operators/BlockingOperatorNextTest.java index e53915d6ae..743a8b3b09 100644 --- a/src/test/java/rx/internal/operators/BlockingOperatorNextTest.java +++ b/src/test/java/rx/internal/operators/BlockingOperatorNextTest.java @@ -15,11 +15,8 @@ */ package rx.internal.operators; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; -import static rx.internal.operators.BlockingOperatorNext.next; +import org.junit.Assert; +import org.junit.Test; import java.util.Iterator; import java.util.NoSuchElementException; @@ -28,19 +25,21 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import org.junit.Assert; -import org.junit.Test; - import rx.Observable; import rx.Subscriber; import rx.exceptions.TestException; -import rx.internal.operators.BlockingOperatorNext; import rx.observables.BlockingObservable; import rx.schedulers.Schedulers; import rx.subjects.BehaviorSubject; import rx.subjects.PublishSubject; import rx.subjects.Subject; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static rx.internal.operators.BlockingOperatorNext.next; + public class BlockingOperatorNextTest { private void fireOnNextInNewThread(final Subject o, final String value) { @@ -83,6 +82,13 @@ public void testNext() { assertTrue(it.hasNext()); assertEquals("two", it.next()); + fireOnNextInNewThread(obs, "three"); + try { + assertEquals("three", it.next()); + } catch (NoSuchElementException e) { + fail("Calling next() without hasNext() should wait for next fire"); + } + obs.onCompleted(); assertFalse(it.hasNext()); try { diff --git a/src/test/java/rx/observables/BlockingObservableTest.java b/src/test/java/rx/observables/BlockingObservableTest.java index 95b0e21ac5..4328461d80 100644 --- a/src/test/java/rx/observables/BlockingObservableTest.java +++ b/src/test/java/rx/observables/BlockingObservableTest.java @@ -15,21 +15,18 @@ */ package rx.observables; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -import java.util.Iterator; -import java.util.NoSuchElementException; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; - import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.mockito.Mock; import org.mockito.MockitoAnnotations; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + import rx.Observable; import rx.Observable.OnSubscribe; import rx.Subscriber; @@ -40,6 +37,12 @@ import rx.schedulers.Schedulers; import rx.subscriptions.Subscriptions; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + public class BlockingObservableTest { @Mock @@ -357,7 +360,7 @@ public void testFirstOrDefault() { @Test public void testFirstOrDefaultWithEmpty() { - BlockingObservable observable = BlockingObservable.from(Observable. empty()); + BlockingObservable observable = BlockingObservable.from(Observable.empty()); assertEquals("default", observable.firstOrDefault("default")); } @@ -411,117 +414,196 @@ public void call() { assertTrue("Timeout means `unsubscribe` is not called", unsubscribe.await(30, TimeUnit.SECONDS)); } + private Action1> singleAction = new Action1>() { + @Override + public void call(final BlockingObservable o) { + o.single(); + } + }; + @Test public void testUnsubscribeFromSingleWhenInterrupted() throws InterruptedException { - new InterruptionTests().assertUnsubscribeIsInvoked("single()", new Action1>() { - @Override - public void call(final BlockingObservable o) { - o.single(); - } - }); + new InterruptionTests().assertUnsubscribeIsInvoked("single()", singleAction); } + @Test + public void testNoInterruptedExceptionWhenInterruptedWhileSingleOnSynchronousObservable() throws InterruptedException { + new InterruptionTests().assertNoInterruptedExceptionWhenSynchronous("single()", singleAction); + } + + private Action1> forEachAction = new Action1>() { + @Override + public void call(final BlockingObservable o) { + o.forEach(new Action1() { + @Override + public void call(final Void aVoid) { + // nothing + } + }); + } + }; + @Test public void testUnsubscribeFromForEachWhenInterrupted() throws InterruptedException { - new InterruptionTests().assertUnsubscribeIsInvoked("forEach()", new Action1>() { - @Override - public void call(final BlockingObservable o) { - o.forEach(new Action1() { - @Override - public void call(final Void aVoid) { - // nothing - } - }); - } - }); + new InterruptionTests().assertUnsubscribeIsInvoked("forEach()", forEachAction); } + @Test + public void testNoInterruptedExceptionWhenInterruptedWhileForEachOnSynchronousObservable() throws InterruptedException { + new InterruptionTests().assertNoInterruptedExceptionWhenSynchronous("forEach()", forEachAction); + } + + private Action1> firstAction = new Action1>() { + @Override + public void call(final BlockingObservable o) { + o.first(); + } + }; + @Test public void testUnsubscribeFromFirstWhenInterrupted() throws InterruptedException { - new InterruptionTests().assertUnsubscribeIsInvoked("first()", new Action1>() { - @Override - public void call(final BlockingObservable o) { - o.first(); - } - }); + new InterruptionTests().assertUnsubscribeIsInvoked("first()", firstAction); } + @Test + public void testNoInterruptedExceptionWhenInterruptedWhileFirstOnSynchronousObservable() throws InterruptedException { + new InterruptionTests().assertNoInterruptedExceptionWhenSynchronous("first()", firstAction); + } + + private Action1> lastAction = new Action1>() { + @Override + public void call(final BlockingObservable o) { + o.last(); + } + }; + @Test public void testUnsubscribeFromLastWhenInterrupted() throws InterruptedException { - new InterruptionTests().assertUnsubscribeIsInvoked("last()", new Action1>() { - @Override - public void call(final BlockingObservable o) { - o.last(); - } - }); + new InterruptionTests().assertUnsubscribeIsInvoked("last()", lastAction); + } + + @Test + public void testNoInterruptedExceptionWhenInterruptedWhileLastOnSynchronousObservable() throws InterruptedException { + new InterruptionTests().assertNoInterruptedExceptionWhenSynchronous("last()", lastAction); } + private Action1> latestAction = new Action1>() { + @Override + public void call(final BlockingObservable o) { + o.latest().iterator().next(); + } + }; + @Test public void testUnsubscribeFromLatestWhenInterrupted() throws InterruptedException { - new InterruptionTests().assertUnsubscribeIsInvoked("latest()", new Action1>() { - @Override - public void call(final BlockingObservable o) { - o.latest().iterator().next(); - } - }); + new InterruptionTests().assertUnsubscribeIsInvoked("latest()", latestAction); } + // NOTE: latest() is intended to be async, so InterruptedException will be thrown even if synchronous + + private Action1> nextAction = new Action1>() { + @Override + public void call(final BlockingObservable o) { + o.next().iterator().next(); + } + }; + @Test public void testUnsubscribeFromNextWhenInterrupted() throws InterruptedException { - new InterruptionTests().assertUnsubscribeIsInvoked("next()", new Action1>() { - @Override - public void call(final BlockingObservable o) { - o.next().iterator().next(); - } - }); + new InterruptionTests().assertUnsubscribeIsInvoked("next()", nextAction); } + // NOTE: next() is intended to be async, so InterruptedException will be thrown even if synchronous + + private Action1> getIteratorAction = new Action1>() { + @Override + public void call(final BlockingObservable o) { + o.getIterator().next(); + } + }; + @Test public void testUnsubscribeFromGetIteratorWhenInterrupted() throws InterruptedException { - new InterruptionTests().assertUnsubscribeIsInvoked("getIterator()", new Action1>() { - @Override - public void call(final BlockingObservable o) { - o.getIterator().next(); - } - }); + new InterruptionTests().assertUnsubscribeIsInvoked("getIterator()", getIteratorAction); } + @Test + public void testNoInterruptedExceptionWhenInterruptedWhileGetIteratorOnSynchronousObservable() throws InterruptedException { + new InterruptionTests().assertNoInterruptedExceptionWhenSynchronous("getIterator()", getIteratorAction); + } + + private Action1> toIterableAction = new Action1>() { + @Override + public void call(final BlockingObservable o) { + o.toIterable().iterator().next(); + } + }; + @Test public void testUnsubscribeFromToIterableWhenInterrupted() throws InterruptedException { - new InterruptionTests().assertUnsubscribeIsInvoked("toIterable()", new Action1>() { - @Override - public void call(final BlockingObservable o) { - o.toIterable().iterator().next(); - } - }); + new InterruptionTests().assertUnsubscribeIsInvoked("toIterable()", toIterableAction); + } + + @Test + public void testNoInterruptedExceptionWhenInterruptedWhileToIterableOnSynchronousObservable() throws InterruptedException { + new InterruptionTests().assertNoInterruptedExceptionWhenSynchronous("toIterable()", toIterableAction); } /** Utilities set for interruption behaviour tests. */ private static class InterruptionTests { private boolean isUnSubscribed; - private RuntimeException error; + private final AtomicReference errorRef = new AtomicReference(); private CountDownLatch latch = new CountDownLatch(1); - private Observable createObservable() { - return Observable.never().doOnUnsubscribe(new Action0() { + private Action0 createOnUnsubscribe() { + return new Action0() { @Override public void call() { isUnSubscribed = true; } - }); + }; + } + + private Observable createNeverObservable() { + return Observable.never().doOnUnsubscribe(createOnUnsubscribe()); } - private void startBlockingAndInterrupt(final Action1> blockingAction) { + private Observable createSynchronousObservable() { + return Observable.from(new Iterable() { + @Override + public Iterator iterator() { + return new Iterator() { + private boolean nextCalled = false; + + @Override + public boolean hasNext() { + return !(nextCalled && Thread.currentThread().isInterrupted()); + } + + @Override + public Void next() { + nextCalled = true; + return null; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("Read-only iterator."); + } + }; + } + }).takeLast(1).doOnUnsubscribe(createOnUnsubscribe()); + } + + private void startBlockingAndInterrupt(final Observable observable, final Action1> blockingAction) { Thread subscriptionThread = new Thread() { @Override public void run() { try { - blockingAction.call(createObservable().toBlocking()); + blockingAction.call(observable.toBlocking()); } catch (RuntimeException e) { - if (!(e.getCause() instanceof InterruptedException)) { - error = e; - } + errorRef.set(e); } latch.countDown(); } @@ -532,14 +614,31 @@ public void run() { void assertUnsubscribeIsInvoked(final String method, final Action1> blockingAction) throws InterruptedException { - startBlockingAndInterrupt(blockingAction); + startBlockingAndInterrupt(createNeverObservable(), blockingAction); assertTrue("Timeout means interruption is not performed", latch.await(30, TimeUnit.SECONDS)); - if (error != null) { - throw error; - } + assertNotNull("InterruptedException is not thrown", getInterruptedExceptionOrNull()); assertTrue("'unsubscribe' is not invoked when thread is interrupted for " + method, isUnSubscribed); } + void assertNoInterruptedExceptionWhenSynchronous(final String method, final Action1> blockingAction) + throws InterruptedException { + startBlockingAndInterrupt(createSynchronousObservable(), blockingAction); + assertTrue("Timeout means interruption is not performed", latch.await(30, TimeUnit.SECONDS)); + assertNull("'InterruptedException' is thrown when observable is synchronous for " + method, getInterruptedExceptionOrNull()); + } + + private InterruptedException getInterruptedExceptionOrNull() { + RuntimeException error = errorRef.get(); + if (error == null) { + return null; + } + Throwable cause = error.getCause(); + if (cause instanceof InterruptedException) { + return (InterruptedException) cause; + } + throw error; + } + } } From bee4f4b68665fabb35c4f405f8e7b872d94174b9 Mon Sep 17 00:00:00 2001 From: Artem Zinnatullin Date: Wed, 29 Jul 2015 21:13:09 +0300 Subject: [PATCH 23/81] Improve performance of NewThreadWorker.tryEnableCancelPolicy(). Disable search for ScheduledExecutorService.setRemoveOnCancelPolicy() on Android API < 21 --- .../internal/schedulers/NewThreadWorker.java | 104 +++++++++++++++--- .../rx/internal/util/PlatformDependent.java | 46 ++++++-- .../schedulers/NewThreadWorkerTest.java | 65 +++++++++++ 3 files changed, 186 insertions(+), 29 deletions(-) create mode 100644 src/test/java/rx/internal/schedulers/NewThreadWorkerTest.java diff --git a/src/main/java/rx/internal/schedulers/NewThreadWorker.java b/src/main/java/rx/internal/schedulers/NewThreadWorker.java index 094c94892f..4c47936871 100644 --- a/src/main/java/rx/internal/schedulers/NewThreadWorker.java +++ b/src/main/java/rx/internal/schedulers/NewThreadWorker.java @@ -27,6 +27,8 @@ import rx.plugins.*; import rx.subscriptions.*; +import static rx.internal.util.PlatformDependent.ANDROID_API_VERSION_IS_NOT_ANDROID; + /** * @warn class description missing */ @@ -39,8 +41,7 @@ public class NewThreadWorker extends Scheduler.Worker implements Subscription { /** Force the use of purge (true/false). */ private static final String PURGE_FORCE_KEY = "rx.scheduler.jdk6.purge-force"; private static final String PURGE_THREAD_PREFIX = "RxSchedulerPurge-"; - /** Forces the use of purge even if setRemoveOnCancelPolicy is available. */ - private static final boolean PURGE_FORCE; + private static final boolean SHOULD_TRY_ENABLE_CANCEL_POLICY; /** The purge frequency in milliseconds. */ public static final int PURGE_FREQUENCY; private static final ConcurrentHashMap EXECUTORS; @@ -48,8 +49,17 @@ public class NewThreadWorker extends Scheduler.Worker implements Subscription { static { EXECUTORS = new ConcurrentHashMap(); PURGE = new AtomicReference(); - PURGE_FORCE = Boolean.getBoolean(PURGE_FORCE_KEY); PURGE_FREQUENCY = Integer.getInteger(FREQUENCY_KEY, 1000); + + // Forces the use of purge even if setRemoveOnCancelPolicy is available + final boolean purgeForce = Boolean.getBoolean(PURGE_FORCE_KEY); + + final int androidApiVersion = PlatformDependent.getAndroidApiVersion(); + + // According to http://developer.android.com/reference/java/util/concurrent/ScheduledThreadPoolExecutor.html#setRemoveOnCancelPolicy(boolean) + // setRemoveOnCancelPolicy available since Android API 21 + SHOULD_TRY_ENABLE_CANCEL_POLICY = !purgeForce + && (androidApiVersion == ANDROID_API_VERSION_IS_NOT_ANDROID || androidApiVersion >= 21); } /** * Registers the given executor service and starts the purge thread if not already started. @@ -85,6 +95,7 @@ public void run() { public static void deregisterExecutor(ScheduledExecutorService service) { EXECUTORS.remove(service); } + /** Purges each registered executor and eagerly evicts shutdown executors. */ static void purgeExecutors() { try { @@ -102,32 +113,89 @@ static void purgeExecutors() { RxJavaPlugins.getInstance().getErrorHandler().handleError(t); } } - - /** + + /** + * Improves performance of {@link #tryEnableCancelPolicy(ScheduledExecutorService)}. + * Also, it works even for inheritance: {@link Method} of base class can be invoked on the instance of child class. + */ + private static volatile Object cachedSetRemoveOnCancelPolicyMethod; + + /** + * Possible value of {@link #cachedSetRemoveOnCancelPolicyMethod} which means that cancel policy is not supported. + */ + private static final Object SET_REMOVE_ON_CANCEL_POLICY_METHOD_NOT_SUPPORTED = new Object(); + + /** * Tries to enable the Java 7+ setRemoveOnCancelPolicy. *

{@code public} visibility reason: called from other package(s) within RxJava. * If the method returns false, the {@link #registerExecutor(ScheduledThreadPoolExecutor)} may * be called to enable the backup option of purging the executors. - * @param exec the executor to call setRemoveOnCaneclPolicy if available. + * @param executor the executor to call setRemoveOnCaneclPolicy if available. * @return true if the policy was successfully enabled */ - public static boolean tryEnableCancelPolicy(ScheduledExecutorService exec) { - if (!PURGE_FORCE) { - for (Method m : exec.getClass().getMethods()) { - if (m.getName().equals("setRemoveOnCancelPolicy") - && m.getParameterTypes().length == 1 - && m.getParameterTypes()[0] == Boolean.TYPE) { - try { - m.invoke(exec, true); - return true; - } catch (Exception ex) { - RxJavaPlugins.getInstance().getErrorHandler().handleError(ex); - } + public static boolean tryEnableCancelPolicy(ScheduledExecutorService executor) { + if (SHOULD_TRY_ENABLE_CANCEL_POLICY) { + final boolean isInstanceOfScheduledThreadPoolExecutor = executor instanceof ScheduledThreadPoolExecutor; + + final Method methodToCall; + + if (isInstanceOfScheduledThreadPoolExecutor) { + final Object localSetRemoveOnCancelPolicyMethod = cachedSetRemoveOnCancelPolicyMethod; + + if (localSetRemoveOnCancelPolicyMethod == SET_REMOVE_ON_CANCEL_POLICY_METHOD_NOT_SUPPORTED) { + return false; + } + + if (localSetRemoveOnCancelPolicyMethod == null) { + Method method = findSetRemoveOnCancelPolicyMethod(executor); + + cachedSetRemoveOnCancelPolicyMethod = method != null + ? method + : SET_REMOVE_ON_CANCEL_POLICY_METHOD_NOT_SUPPORTED; + + methodToCall = method; + } else { + methodToCall = (Method) localSetRemoveOnCancelPolicyMethod; + } + } else { + methodToCall = findSetRemoveOnCancelPolicyMethod(executor); + } + + if (methodToCall != null) { + try { + methodToCall.invoke(executor, true); + return true; + } catch (Exception e) { + RxJavaPlugins.getInstance().getErrorHandler().handleError(e); } } } + return false; } + + /** + * Tries to find {@code "setRemoveOnCancelPolicy(boolean)"} method in the class of passed executor. + * + * @param executor whose class will be used to search for required method. + * @return {@code "setRemoveOnCancelPolicy(boolean)"} {@link Method} + * or {@code null} if required {@link Method} was not found. + */ + static Method findSetRemoveOnCancelPolicyMethod(ScheduledExecutorService executor) { + // The reason for the loop is to avoid NoSuchMethodException being thrown on JDK 6 + // which is more costly than looping through ~70 methods. + for (final Method method : executor.getClass().getMethods()) { + if (method.getName().equals("setRemoveOnCancelPolicy")) { + final Class[] parameterTypes = method.getParameterTypes(); + + if (parameterTypes.length == 1 && parameterTypes[0] == Boolean.TYPE) { + return method; + } + } + } + + return null; + } /* package */ public NewThreadWorker(ThreadFactory threadFactory) { diff --git a/src/main/java/rx/internal/util/PlatformDependent.java b/src/main/java/rx/internal/util/PlatformDependent.java index c6b7f3c28f..614e327a7e 100644 --- a/src/main/java/rx/internal/util/PlatformDependent.java +++ b/src/main/java/rx/internal/util/PlatformDependent.java @@ -20,31 +20,55 @@ /** * Allow platform dependent logic such as checks for Android. - * + * * Modeled after Netty with some code copy/pasted from: https://github.com/netty/netty/blob/master/common/src/main/java/io/netty/util/internal/PlatformDependent.java */ public final class PlatformDependent { - private static final boolean IS_ANDROID = isAndroid0(); + /** + * Possible value of {@link #getAndroidApiVersion()} which means that the current platform is not Android. + */ + public static final int ANDROID_API_VERSION_IS_NOT_ANDROID = 0; + + private static final int ANDROID_API_VERSION = resolveAndroidApiVersion(); + + private static final boolean IS_ANDROID = ANDROID_API_VERSION != ANDROID_API_VERSION_IS_NOT_ANDROID; /** - * Returns {@code true} if and only if the current platform is Android + * Returns {@code true} if and only if the current platform is Android. */ public static boolean isAndroid() { return IS_ANDROID; } - private static boolean isAndroid0() { - boolean android; + /** + * Returns version of Android API. + * + * @return version of Android API or {@link #ANDROID_API_VERSION_IS_NOT_ANDROID } if version + * can not be resolved or if current platform is not Android. + */ + public static int getAndroidApiVersion() { + return ANDROID_API_VERSION; + } + + /** + * Resolves version of Android API. + * + * @return version of Android API or {@link #ANDROID_API_VERSION_IS_NOT_ANDROID} if version can not be resolved + * or if the current platform is not Android. + * @see Documentation + */ + private static int resolveAndroidApiVersion() { try { - Class.forName("android.app.Application", false, getSystemClassLoader()); - android = true; + return (Integer) Class + .forName("android.os.Build$VERSION", true, getSystemClassLoader()) + .getField("SDK_INT") + .get(null); } catch (Exception e) { - // Failed to load the class uniquely available in Android. - android = false; + // Can not resolve version of Android API, maybe current platform is not Android + // or API of resolving current Version of Android API has changed in some release of Android + return ANDROID_API_VERSION_IS_NOT_ANDROID; } - - return android; } /** diff --git a/src/test/java/rx/internal/schedulers/NewThreadWorkerTest.java b/src/test/java/rx/internal/schedulers/NewThreadWorkerTest.java new file mode 100644 index 0000000000..c0d6f93dda --- /dev/null +++ b/src/test/java/rx/internal/schedulers/NewThreadWorkerTest.java @@ -0,0 +1,65 @@ +package rx.internal.schedulers; + +import org.junit.Test; + +import java.lang.reflect.Field; +import java.lang.reflect.Method; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.atomic.AtomicBoolean; + +import static java.lang.reflect.Modifier.FINAL; +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +public class NewThreadWorkerTest { + + @Test + public void findSetRemoveOnCancelPolicyMethodShouldFindMethod() { + ScheduledExecutorService executor = spy(new ScheduledThreadPoolExecutor(1)); + Method setRemoveOnCancelPolicyMethod = NewThreadWorker.findSetRemoveOnCancelPolicyMethod(executor); + + assertNotNull(setRemoveOnCancelPolicyMethod); + assertEquals("setRemoveOnCancelPolicy", setRemoveOnCancelPolicyMethod.getName()); + assertEquals(1, setRemoveOnCancelPolicyMethod.getParameterTypes().length); + assertEquals(Boolean.TYPE, setRemoveOnCancelPolicyMethod.getParameterTypes()[0]); + verifyZeroInteractions(executor); + } + + @Test + public void findSetRemoveOnCancelPolicyMethodShouldNotFindMethod() { + ScheduledExecutorService executor = mock(ScheduledExecutorService.class); + + Method setRemoveOnCancelPolicyMethod = NewThreadWorker.findSetRemoveOnCancelPolicyMethod(executor); + assertNull(setRemoveOnCancelPolicyMethod); + verifyZeroInteractions(executor); + } + + private static abstract class ScheduledExecutorServiceWithSetRemoveOnCancelPolicy implements ScheduledExecutorService { + // Just declaration of required method to allow run tests on JDK 6 + public void setRemoveOnCancelPolicy(@SuppressWarnings("UnusedParameters") boolean value) {} + } + + @Test + public void tryEnableCancelPolicyShouldInvokeMethodOnExecutor() { + ScheduledExecutorServiceWithSetRemoveOnCancelPolicy executor + = mock(ScheduledExecutorServiceWithSetRemoveOnCancelPolicy.class); + + boolean result = NewThreadWorker.tryEnableCancelPolicy(executor); + + assertTrue(result); + verify(executor).setRemoveOnCancelPolicy(true); + verifyNoMoreInteractions(executor); + } + + @Test + public void tryEnableCancelPolicyShouldNotInvokeMethodOnExecutor() { + // This executor does not have setRemoveOnCancelPolicy method + ScheduledExecutorService executor = mock(ScheduledExecutorService.class); + + boolean result = NewThreadWorker.tryEnableCancelPolicy(executor); + + assertFalse(result); + verifyZeroInteractions(executor); + } +} From 8b5cbb05bb0a9a726b162eaa2d33dec0abb7252d Mon Sep 17 00:00:00 2001 From: akarnokd Date: Mon, 3 Aug 2015 21:26:22 +0200 Subject: [PATCH 24/81] Fix retry with predicate ignoring backpressure. --- src/main/java/rx/Observable.java | 2 + .../operators/OperatorRetryWithPredicate.java | 116 ++++++++++-------- .../OperatorRetryWithPredicateTest.java | 41 ++++++- 3 files changed, 105 insertions(+), 54 deletions(-) diff --git a/src/main/java/rx/Observable.java b/src/main/java/rx/Observable.java index 0aafecbf79..a6432a89a0 100644 --- a/src/main/java/rx/Observable.java +++ b/src/main/java/rx/Observable.java @@ -6576,6 +6576,8 @@ public final Observable retry(final long count) { *

* *

+ *
Backpressure Support:
+ *
This operator honors backpressure. *
Scheduler:
*
{@code retry} operates by default on the {@code trampoline} {@link Scheduler}.
*
diff --git a/src/main/java/rx/internal/operators/OperatorRetryWithPredicate.java b/src/main/java/rx/internal/operators/OperatorRetryWithPredicate.java index 24beeec2a0..bdfcd3dbeb 100644 --- a/src/main/java/rx/internal/operators/OperatorRetryWithPredicate.java +++ b/src/main/java/rx/internal/operators/OperatorRetryWithPredicate.java @@ -16,11 +16,14 @@ package rx.internal.operators; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; + import rx.Observable; +import rx.Producer; import rx.Scheduler; import rx.Subscriber; import rx.functions.Action0; import rx.functions.Func2; +import rx.internal.producers.ProducerArbiter; import rx.schedulers.Schedulers; import rx.subscriptions.SerialSubscription; @@ -38,8 +41,9 @@ public Subscriber> call(final Subscriber child) final SerialSubscription serialSubscription = new SerialSubscription(); // add serialSubscription so it gets unsubscribed if child is unsubscribed child.add(serialSubscription); - - return new SourceSubscriber(child, predicate, inner, serialSubscription); + ProducerArbiter pa = new ProducerArbiter(); + child.setProducer(pa); + return new SourceSubscriber(child, predicate, inner, serialSubscription, pa); } static final class SourceSubscriber extends Subscriber> { @@ -47,79 +51,89 @@ static final class SourceSubscriber extends Subscriber> { final Func2 predicate; final Scheduler.Worker inner; final SerialSubscription serialSubscription; + final ProducerArbiter pa; volatile int attempts; @SuppressWarnings("rawtypes") static final AtomicIntegerFieldUpdater ATTEMPTS_UPDATER = AtomicIntegerFieldUpdater.newUpdater(SourceSubscriber.class, "attempts"); - public SourceSubscriber(Subscriber child, final Func2 predicate, Scheduler.Worker inner, - SerialSubscription serialSubscription) { + public SourceSubscriber(Subscriber child, + final Func2 predicate, + Scheduler.Worker inner, + SerialSubscription serialSubscription, + ProducerArbiter pa) { this.child = child; this.predicate = predicate; this.inner = inner; this.serialSubscription = serialSubscription; + this.pa = pa; } @Override - public void onCompleted() { - // ignore as we expect a single nested Observable - } + public void onCompleted() { + // ignore as we expect a single nested Observable + } - @Override - public void onError(Throwable e) { - child.onError(e); - } + @Override + public void onError(Throwable e) { + child.onError(e); + } - @Override - public void onNext(final Observable o) { - inner.schedule(new Action0() { + @Override + public void onNext(final Observable o) { + inner.schedule(new Action0() { - @Override - public void call() { - final Action0 _self = this; - ATTEMPTS_UPDATER.incrementAndGet(SourceSubscriber.this); + @Override + public void call() { + final Action0 _self = this; + ATTEMPTS_UPDATER.incrementAndGet(SourceSubscriber.this); - // new subscription each time so if it unsubscribes itself it does not prevent retries - // by unsubscribing the child subscription - Subscriber subscriber = new Subscriber() { - boolean done; - @Override - public void onCompleted() { - if (!done) { - done = true; - child.onCompleted(); - } + // new subscription each time so if it unsubscribes itself it does not prevent retries + // by unsubscribing the child subscription + Subscriber subscriber = new Subscriber() { + boolean done; + @Override + public void onCompleted() { + if (!done) { + done = true; + child.onCompleted(); } + } - @Override - public void onError(Throwable e) { - if (!done) { - done = true; - if (predicate.call(attempts, e) && !inner.isUnsubscribed()) { - // retry again - inner.schedule(_self); - } else { - // give up and pass the failure - child.onError(e); - } + @Override + public void onError(Throwable e) { + if (!done) { + done = true; + if (predicate.call(attempts, e) && !inner.isUnsubscribed()) { + // retry again + inner.schedule(_self); + } else { + // give up and pass the failure + child.onError(e); } } + } - @Override - public void onNext(T v) { - if (!done) { - child.onNext(v); - } + @Override + public void onNext(T v) { + if (!done) { + child.onNext(v); + pa.produced(1); } + } - }; - // register this Subscription (and unsubscribe previous if exists) - serialSubscription.set(subscriber); - o.unsafeSubscribe(subscriber); - } - }); - } + @Override + public void setProducer(Producer p) { + pa.setProducer(p); + } + }; + // register this Subscription (and unsubscribe previous if exists) + serialSubscription.set(subscriber); + o.unsafeSubscribe(subscriber); + } + }); + } } } diff --git a/src/test/java/rx/internal/operators/OperatorRetryWithPredicateTest.java b/src/test/java/rx/internal/operators/OperatorRetryWithPredicateTest.java index 76461e3ddf..df878de13a 100644 --- a/src/test/java/rx/internal/operators/OperatorRetryWithPredicateTest.java +++ b/src/test/java/rx/internal/operators/OperatorRetryWithPredicateTest.java @@ -20,20 +20,27 @@ import static org.mockito.Mockito.*; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.*; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import org.junit.Test; import org.mockito.InOrder; -import rx.*; +import rx.Observable; import rx.Observable.OnSubscribe; +import rx.Observer; +import rx.Subscriber; +import rx.Subscription; import rx.exceptions.TestException; -import rx.functions.*; +import rx.functions.Action1; +import rx.functions.Func1; +import rx.functions.Func2; import rx.observers.TestSubscriber; import rx.subjects.PublishSubject; @@ -360,4 +367,32 @@ public void call(Long t) { }}); assertEquals(Arrays.asList(1L,1L,2L,3L), list); } + @Test + public void testBackpressure() { + final List requests = new ArrayList(); + + Observable source = Observable + .just(1) + .concatWith(Observable.error(new TestException())) + .doOnRequest(new Action1() { + @Override + public void call(Long t) { + requests.add(t); + } + }); + + TestSubscriber ts = TestSubscriber.create(3); + source + .retry(new Func2() { + @Override + public Boolean call(Integer t1, Throwable t2) { + return t1 < 3; + } + }).subscribe(ts); + + assertEquals(Arrays.asList(3L, 2L, 1L), requests); + ts.assertValues(1, 1, 1); + ts.assertNotCompleted(); + ts.assertNoErrors(); + } } From 04c03fb2aa07e5ae0a7d8e3e9c2f0a35194e7465 Mon Sep 17 00:00:00 2001 From: Dave Moten Date: Thu, 23 Jul 2015 16:21:41 +1000 Subject: [PATCH 25/81] add backpressure to OperatorMaterialize --- .../operators/OperatorMaterialize.java | 133 +++++++++++++++--- .../operators/OperatorMaterializeTest.java | 114 ++++++++++++++- 2 files changed, 227 insertions(+), 20 deletions(-) diff --git a/src/main/java/rx/internal/operators/OperatorMaterialize.java b/src/main/java/rx/internal/operators/OperatorMaterialize.java index bd5771747c..e074cd5816 100644 --- a/src/main/java/rx/internal/operators/OperatorMaterialize.java +++ b/src/main/java/rx/internal/operators/OperatorMaterialize.java @@ -15,8 +15,11 @@ */ package rx.internal.operators; +import java.util.concurrent.atomic.AtomicLongFieldUpdater; + import rx.Notification; import rx.Observable.Operator; +import rx.Producer; import rx.Subscriber; import rx.plugins.RxJavaPlugins; @@ -29,41 +32,137 @@ * See here for the Microsoft Rx equivalent. */ public final class OperatorMaterialize implements Operator, T> { + /** Lazy initialization via inner-class holder. */ private static final class Holder { /** A singleton instance. */ static final OperatorMaterialize INSTANCE = new OperatorMaterialize(); } + /** * @return a singleton instance of this stateless operator. */ @SuppressWarnings("unchecked") public static OperatorMaterialize instance() { - return (OperatorMaterialize)Holder.INSTANCE; + return (OperatorMaterialize) Holder.INSTANCE; } - private OperatorMaterialize() { } + + private OperatorMaterialize() { + } + @Override public Subscriber call(final Subscriber> child) { - return new Subscriber(child) { - + final ParentSubscriber parent = new ParentSubscriber(child); + child.add(parent); + child.setProducer(new Producer() { @Override - public void onCompleted() { - child.onNext(Notification. createOnCompleted()); - child.onCompleted(); + public void request(long n) { + if (n > 0) { + parent.requestMore(n); + } } + }); + return parent; + } - @Override - public void onError(Throwable e) { - RxJavaPlugins.getInstance().getErrorHandler().handleError(e); - child.onNext(Notification. createOnError(e)); - child.onCompleted(); - } + private static class ParentSubscriber extends Subscriber { - @Override - public void onNext(T t) { - child.onNext(Notification. createOnNext(t)); + private final Subscriber> child; + + private volatile Notification terminalNotification; + + // guarded by this + private boolean busy = false; + // guarded by this + private boolean missed = false; + + private volatile long requested; + @SuppressWarnings("rawtypes") + private static final AtomicLongFieldUpdater REQUESTED = AtomicLongFieldUpdater + .newUpdater(ParentSubscriber.class, "requested"); + + ParentSubscriber(Subscriber> child) { + this.child = child; + } + + @Override + public void onStart() { + request(0); + } + + void requestMore(long n) { + BackpressureUtils.getAndAddRequest(REQUESTED, this, n); + request(n); + drain(); + } + + @Override + public void onCompleted() { + terminalNotification = Notification.createOnCompleted(); + drain(); + } + + @Override + public void onError(Throwable e) { + terminalNotification = Notification.createOnError(e); + RxJavaPlugins.getInstance().getErrorHandler().handleError(e); + drain(); + } + + @Override + public void onNext(T t) { + child.onNext(Notification.createOnNext(t)); + decrementRequested(); + } + + private void decrementRequested() { + // atomically decrement requested + while (true) { + long r = requested; + if (r == Long.MAX_VALUE) { + // don't decrement if unlimited requested + return; + } else if (REQUESTED.compareAndSet(this, r, r - 1)) { + return; + } } + } - }; + private void drain() { + synchronized (this) { + if (busy) { + // set flag to force extra loop if drain loop running + missed = true; + return; + } + } + // drain loop + while (!child.isUnsubscribed()) { + Notification tn; + tn = terminalNotification; + if (tn != null) { + if (requested > 0) { + // allow tn to be GC'd after the onNext call + terminalNotification = null; + // emit the terminal notification + child.onNext(tn); + if (!child.isUnsubscribed()) { + child.onCompleted(); + } + // note that we leave busy=true here + // which will prevent further drains + return; + } + } + // continue looping if drain() was called while in + // this loop + synchronized (this) { + if (!missed) { + busy = false; + return; + } + } + } + } } } diff --git a/src/test/java/rx/internal/operators/OperatorMaterializeTest.java b/src/test/java/rx/internal/operators/OperatorMaterializeTest.java index a900da61d6..511a79ed54 100644 --- a/src/test/java/rx/internal/operators/OperatorMaterializeTest.java +++ b/src/test/java/rx/internal/operators/OperatorMaterializeTest.java @@ -19,6 +19,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import java.util.Arrays; import java.util.List; import java.util.Vector; import java.util.concurrent.ExecutionException; @@ -28,13 +29,18 @@ import rx.Notification; import rx.Observable; import rx.Subscriber; +import rx.functions.Action1; +import rx.observers.TestSubscriber; +import rx.schedulers.Schedulers; public class OperatorMaterializeTest { @Test public void testMaterialize1() { - // null will cause onError to be triggered before "three" can be returned - final TestAsyncErrorObservable o1 = new TestAsyncErrorObservable("one", "two", null, "three"); + // null will cause onError to be triggered before "three" can be + // returned + final TestAsyncErrorObservable o1 = new TestAsyncErrorObservable("one", "two", null, + "three"); TestObserver Observer = new TestObserver(); Observable> m = Observable.create(o1).materialize(); @@ -53,7 +59,8 @@ public void testMaterialize1() { assertTrue(Observer.notifications.get(0).isOnNext()); assertEquals("two", Observer.notifications.get(1).getValue()); assertTrue(Observer.notifications.get(1).isOnNext()); - assertEquals(NullPointerException.class, Observer.notifications.get(2).getThrowable().getClass()); + assertEquals(NullPointerException.class, Observer.notifications.get(2).getThrowable() + .getClass()); assertTrue(Observer.notifications.get(2).isOnError()); } @@ -93,6 +100,107 @@ public void testMultipleSubscribes() throws InterruptedException, ExecutionExcep assertEquals(3, m.toList().toBlocking().toFuture().get().size()); } + @Test + public void testBackpressureOnEmptyStream() { + TestSubscriber> ts = TestSubscriber.create(0); + Observable. empty().materialize().subscribe(ts); + ts.assertNoValues(); + ts.requestMore(1); + ts.assertValueCount(1); + assertTrue(ts.getOnNextEvents().get(0).isOnCompleted()); + ts.assertCompleted(); + } + + @Test + public void testBackpressureNoError() { + TestSubscriber> ts = TestSubscriber.create(0); + Observable.just(1, 2, 3).materialize().subscribe(ts); + ts.assertNoValues(); + ts.requestMore(1); + ts.assertValueCount(1); + ts.requestMore(2); + ts.assertValueCount(3); + ts.requestMore(1); + ts.assertValueCount(4); + ts.assertCompleted(); + } + + @Test + public void testBackpressureNoErrorAsync() throws InterruptedException { + TestSubscriber> ts = TestSubscriber.create(0); + Observable.just(1, 2, 3) + .materialize() + .subscribeOn(Schedulers.computation()) + .subscribe(ts); + Thread.sleep(100); + ts.assertNoValues(); + ts.requestMore(1); + Thread.sleep(100); + ts.assertValueCount(1); + ts.requestMore(2); + Thread.sleep(100); + ts.assertValueCount(3); + ts.requestMore(1); + Thread.sleep(100); + ts.assertValueCount(4); + ts.assertCompleted(); + } + + @Test + public void testBackpressureWithError() { + TestSubscriber> ts = TestSubscriber.create(0); + Observable. error(new IllegalArgumentException()).materialize().subscribe(ts); + ts.assertNoValues(); + ts.requestMore(1); + ts.assertValueCount(1); + ts.assertCompleted(); + } + + @Test + public void testBackpressureWithEmissionThenError() { + TestSubscriber> ts = TestSubscriber.create(0); + IllegalArgumentException ex = new IllegalArgumentException(); + Observable.from(Arrays.asList(1)).concatWith(Observable. error(ex)).materialize() + .subscribe(ts); + ts.assertNoValues(); + ts.requestMore(1); + ts.assertValueCount(1); + assertTrue(ts.getOnNextEvents().get(0).hasValue()); + ts.requestMore(1); + ts.assertValueCount(2); + assertTrue(ts.getOnNextEvents().get(1).isOnError()); + assertTrue(ex == ts.getOnNextEvents().get(1).getThrowable()); + ts.assertCompleted(); + } + + @Test + public void testWithCompletionCausingError() { + TestSubscriber> ts = TestSubscriber.create(); + final RuntimeException ex = new RuntimeException("boo"); + Observable.empty().materialize().doOnNext(new Action1() { + @Override + public void call(Object t) { + throw ex; + } + }).subscribe(ts); + ts.assertError(ex); + ts.assertNoValues(); + ts.assertTerminalEvent(); + } + + @Test + public void testUnsubscribeJustBeforeCompletionNotificationShouldPreventThatNotificationArriving() { + TestSubscriber> ts = TestSubscriber.create(0); + IllegalArgumentException ex = new IllegalArgumentException(); + Observable.empty().materialize() + .subscribe(ts); + ts.assertNoValues(); + ts.unsubscribe(); + ts.requestMore(1); + ts.assertNoValues(); + ts.assertUnsubscribed(); + } + private static class TestObserver extends Subscriber> { boolean onCompleted = false; From 34dfe3b4ffe270d5016add4b4fb3a24354e3a80b Mon Sep 17 00:00:00 2001 From: David Gross Date: Thu, 6 Aug 2015 09:48:29 -0700 Subject: [PATCH 26/81] Add links to page that explains The Observable Contract --- src/main/java/rx/Observable.java | 8 +++++--- src/main/java/rx/internal/util/RxRingBuffer.java | 5 +++-- src/main/java/rx/observables/BlockingObservable.java | 2 +- src/main/java/rx/observers/SafeSubscriber.java | 2 +- src/main/java/rx/subjects/SerializedSubject.java | 4 ++-- src/main/java/rx/subjects/Subject.java | 8 +++++--- 6 files changed, 17 insertions(+), 12 deletions(-) diff --git a/src/main/java/rx/Observable.java b/src/main/java/rx/Observable.java index a6432a89a0..17937609a0 100644 --- a/src/main/java/rx/Observable.java +++ b/src/main/java/rx/Observable.java @@ -6830,8 +6830,8 @@ public final Observable scan(R initialValue, Func2 accum } /** - * Forces an Observable's emissions and notifications to be serialized and for it to obey the Rx contract - * in other ways. + * Forces an Observable's emissions and notifications to be serialized and for it to obey + * the Observable contract in other ways. *

* It is possible for an Observable to invoke its Subscribers' methods asynchronously, perhaps from * different threads. This could make such an Observable poorly-behaved, in that it might try to invoke @@ -7672,7 +7672,9 @@ public void onNext(T t) { * error handling, unsubscribe, or execution hooks. *

* Use this only for implementing an {@link Operator} that requires nested subscriptions. For other - * purposes, use {@link #subscribe(Subscriber)} which ensures the Rx contract and other functionality. + * purposes, use {@link #subscribe(Subscriber)} which ensures + * the Observable contract and other + * functionality. *

*
Scheduler:
*
{@code unsafeSubscribe} does not operate by default on a particular {@link Scheduler}.
diff --git a/src/main/java/rx/internal/util/RxRingBuffer.java b/src/main/java/rx/internal/util/RxRingBuffer.java index 7498b445be..f038b2deec 100644 --- a/src/main/java/rx/internal/util/RxRingBuffer.java +++ b/src/main/java/rx/internal/util/RxRingBuffer.java @@ -26,8 +26,9 @@ import rx.internal.util.unsafe.UnsafeAccess; /** - * This assumes Spsc or Spmc usage. This means only a single producer calling the on* methods. This is the Rx contract of an Observer. - * Concurrent invocations of on* methods will not be thread-safe. + * This assumes Spsc or Spmc usage. This means only a single producer calling the on* methods. This is the Rx + * contract of an Observer (see http://reactivex.io/documentation/contract.html). Concurrent invocations of + * on* methods will not be thread-safe. */ public class RxRingBuffer implements Subscription { diff --git a/src/main/java/rx/observables/BlockingObservable.java b/src/main/java/rx/observables/BlockingObservable.java index 4f4fdda412..7eced68981 100644 --- a/src/main/java/rx/observables/BlockingObservable.java +++ b/src/main/java/rx/observables/BlockingObservable.java @@ -96,7 +96,7 @@ public void forEach(final Action1 onNext) { /* * Use 'subscribe' instead of 'unsafeSubscribe' for Rx contract behavior - * as this is the final subscribe in the chain. + * (see http://reactivex.io/documentation/contract.html) as this is the final subscribe in the chain. */ Subscription subscription = o.subscribe(new Subscriber() { @Override diff --git a/src/main/java/rx/observers/SafeSubscriber.java b/src/main/java/rx/observers/SafeSubscriber.java index c81cd4fbff..0181887c34 100644 --- a/src/main/java/rx/observers/SafeSubscriber.java +++ b/src/main/java/rx/observers/SafeSubscriber.java @@ -26,7 +26,7 @@ /** * {@code SafeSubscriber} is a wrapper around {@code Subscriber} that ensures that the {@code Subscriber} - * complies with the Rx contract. + * complies with the Observable contract. *

* The following is taken from the Rx Design Guidelines * document: diff --git a/src/main/java/rx/subjects/SerializedSubject.java b/src/main/java/rx/subjects/SerializedSubject.java index baaf50b8d4..edf4caeefe 100644 --- a/src/main/java/rx/subjects/SerializedSubject.java +++ b/src/main/java/rx/subjects/SerializedSubject.java @@ -24,8 +24,8 @@ *

* When you use an ordinary {@link Subject} as a {@link Subscriber}, you must take care not to call its * {@link Subscriber#onNext} method (or its other {@code on} methods) from multiple threads, as this could lead - * to non-serialized calls, which violates the Observable contract and creates an ambiguity in the resulting - * Subject. + * to non-serialized calls, which violates the + * Observable contract and creates an ambiguity in the resulting Subject. *

* To protect a {@code Subject} from this danger, you can convert it into a {@code SerializedSubject} with code * like the following: diff --git a/src/main/java/rx/subjects/Subject.java b/src/main/java/rx/subjects/Subject.java index 4e5db6b770..075dfe8e93 100644 --- a/src/main/java/rx/subjects/Subject.java +++ b/src/main/java/rx/subjects/Subject.java @@ -40,10 +40,12 @@ protected Subject(OnSubscribe onSubscribe) { *

* When you use an ordinary {@link Subject} as a {@link Subscriber}, you must take care not to call its * {@link Subscriber#onNext} method (or its other {@code on} methods) from multiple threads, as this could - * lead to non-serialized calls, which violates the Observable contract and creates an ambiguity in the resulting Subject. + * lead to non-serialized calls, which violates + * the Observable contract and creates an + * ambiguity in the resulting Subject. *

- * To protect a {@code Subject} from this danger, you can convert it into a {@code SerializedSubject} with code - * like the following: + * To protect a {@code Subject} from this danger, you can convert it into a {@code SerializedSubject} with + * code like the following: *

{@code
      * mySafeSubject = myUnsafeSubject.toSerialized();
      * }
From 86f071cdf6a8df7429772b101798c375ea5f7261 Mon Sep 17 00:00:00 2001 From: Aaron Tull Date: Thu, 16 Jul 2015 00:59:11 -0500 Subject: [PATCH 27/81] Implemented Observable.x(ConversionFunc) to allow external extensions to Observables. --- src/main/java/rx/Observable.java | 34 ++- .../java/rx/ObservableConversionTest.java | 234 ++++++++++++++++++ src/test/java/rx/ObservableTests.java | 16 +- 3 files changed, 276 insertions(+), 8 deletions(-) create mode 100644 src/test/java/rx/ObservableConversionTest.java diff --git a/src/main/java/rx/Observable.java b/src/main/java/rx/Observable.java index 17937609a0..246e54f023 100644 --- a/src/main/java/rx/Observable.java +++ b/src/main/java/rx/Observable.java @@ -109,6 +109,23 @@ public interface Operator extends Func1, Subscriber< // cover for generics insanity } + /** + * Passes all emitted values from {@code this} Observable to the provided {@link ConversionFunc} to be + * collected and returned as a single value. Note that it is legal for a {@link ConversionFunc} to + * return an Observable (enabling chaining). + * + * @param conversion a function that converts from this {@code Observable} to an {@code R} + * @return an instance of R created by the provided Conversion + */ + @Experimental + public R x(Func1, ? extends R> conversion) { + return conversion.call(new OnSubscribe() { + @Override + public void call(Subscriber subscriber) { + subscriber.add(Observable.subscribe(subscriber, Observable.this)); + }}); + } + /** * Lifts a function to the current Observable and returns a new Observable that when subscribed to will pass * the values of the current Observable through the Operator function. @@ -127,17 +144,17 @@ public interface Operator extends Func1, Subscriber< *
{@code lift} does not operate by default on a particular {@link Scheduler}.
*
* - * @param lift the Operator that implements the Observable-operating function to be applied to the source + * @param operator the Operator that implements the Observable-operating function to be applied to the source * Observable * @return an Observable that is the result of applying the lifted Operator to the source Observable * @see RxJava wiki: Implementing Your Own Operators */ - public final Observable lift(final Operator lift) { + public final Observable lift(final Operator operator) { return new Observable(new OnSubscribe() { @Override public void call(Subscriber o) { try { - Subscriber st = hook.onLift(lift).call(o); + Subscriber st = hook.onLift(operator).call(o); try { // new Subscriber created and being subscribed with so 'onStart' it st.onStart(); @@ -163,7 +180,6 @@ public void call(Subscriber o) { }); } - /** * Transform an Observable by applying a particular Transformer function to it. *

@@ -7752,11 +7768,15 @@ public final Subscription unsafeSubscribe(Subscriber subscriber) { * @see ReactiveX operators documentation: Subscribe */ public final Subscription subscribe(Subscriber subscriber) { - // validate and proceed + return Observable.subscribe(subscriber, this); + } + + private static Subscription subscribe(Subscriber subscriber, Observable observable) { + // validate and proceed if (subscriber == null) { throw new IllegalArgumentException("observer can not be null"); } - if (onSubscribe == null) { + if (observable.onSubscribe == null) { throw new IllegalStateException("onSubscribe function can not be null."); /* * the subscribe function can also be overridden but generally that's not the appropriate approach @@ -7780,7 +7800,7 @@ public final Subscription subscribe(Subscriber subscriber) { // The code below is exactly the same an unsafeSubscribe but not used because it would add a sigificent depth to alreay huge call stacks. try { // allow the hook to intercept and/or decorate - hook.onSubscribeStart(this, onSubscribe).call(subscriber); + hook.onSubscribeStart(observable, observable.onSubscribe).call(subscriber); return hook.onSubscribeReturn(subscriber); } catch (Throwable e) { // special handling for certain Throwable/Error/Exception types diff --git a/src/test/java/rx/ObservableConversionTest.java b/src/test/java/rx/ObservableConversionTest.java new file mode 100644 index 0000000000..543c44780b --- /dev/null +++ b/src/test/java/rx/ObservableConversionTest.java @@ -0,0 +1,234 @@ +package rx; + +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +import static junit.framework.Assert.*; + +import org.junit.Test; + +import rx.Observable.OnSubscribe; +import rx.Observable.Operator; +import rx.exceptions.OnErrorNotImplementedException; +import rx.functions.Func1; +import rx.functions.Func2; +import rx.internal.operators.OperatorFilter; +import rx.internal.operators.OperatorMap; +import rx.observers.TestSubscriber; +import rx.schedulers.Schedulers; + +public class ObservableConversionTest { + + public static class Cylon {} + + public static class Jail { + Object cylon; + + Jail(Object cylon) { + this.cylon = cylon; + } + } + + public static class CylonDetectorObservable { + protected OnSubscribe onSubscribe; + + public static CylonDetectorObservable create(OnSubscribe onSubscribe) { + return new CylonDetectorObservable(onSubscribe); + } + + protected CylonDetectorObservable(OnSubscribe onSubscribe) { + this.onSubscribe = onSubscribe; + } + + public void subscribe(Subscriber subscriber) { + onSubscribe.call(subscriber); + } + + public CylonDetectorObservable lift(Operator operator) { + return x(new RobotConversionFunc(operator)); + } + + public O x(Func1, O> operator) { + return operator.call(onSubscribe); + } + + public CylonDetectorObservable compose(Func1, CylonDetectorObservable> transformer) { + return transformer.call(this); + } + + public final CylonDetectorObservable beep(Func1 predicate) { + return lift(new OperatorFilter(predicate)); + } + + public final CylonDetectorObservable boop(Func1 func) { + return lift(new OperatorMap(func)); + } + + public CylonDetectorObservable DESTROY() { + return boop(new Func1() { + @Override + public String call(T t) { + Object cylon = ((Jail) t).cylon; + throwOutTheAirlock(cylon); + if (t instanceof Jail) { + String name = cylon.toString(); + return "Cylon '" + name + "' has been destroyed"; + } + else { + return "Cylon 'anonymous' has been destroyed"; + } + }}); + } + + private static void throwOutTheAirlock(Object cylon) { + // ... + } + } + + public static class RobotConversionFunc implements Func1, CylonDetectorObservable> { + private Operator operator; + + public RobotConversionFunc(Operator operator) { + this.operator = operator; + } + + @Override + public CylonDetectorObservable call(final OnSubscribe onSubscribe) { + return CylonDetectorObservable.create(new OnSubscribe() { + @Override + public void call(Subscriber o) { + try { + Subscriber st = operator.call(o); + try { + st.onStart(); + onSubscribe.call(st); + } catch (OnErrorNotImplementedException e) { + throw e; + } catch (Throwable e) { + st.onError(e); + } + } catch (OnErrorNotImplementedException e) { + throw e; + } catch (Throwable e) { + o.onError(e); + } + + }}); + } + } + + public static class ConvertToCylonDetector implements Func1, CylonDetectorObservable> { + @Override + public CylonDetectorObservable call(final OnSubscribe onSubscribe) { + return CylonDetectorObservable.create(onSubscribe); + } + } + + public static class ConvertToObservable implements Func1, Observable> { + @Override + public Observable call(final OnSubscribe onSubscribe) { + return Observable.create(onSubscribe); + } + } + + @Test + public void testConversionBetweenObservableClasses() { + final TestSubscriber subscriber = new TestSubscriber(new Subscriber(){ + + @Override + public void onCompleted() { + System.out.println("Complete"); + } + + @Override + public void onError(Throwable e) { + System.out.println("error: " + e.getMessage()); + e.printStackTrace(); + } + + @Override + public void onNext(String t) { + System.out.println(t); + }}); + List crewOfBattlestarGalactica = Arrays.asList(new Object[] {"William Adama", "Laura Roslin", "Lee Adama", new Cylon()}); + Observable.from(crewOfBattlestarGalactica) + .x(new ConvertToCylonDetector()) + .beep(new Func1(){ + @Override + public Boolean call(Object t) { + return t instanceof Cylon; + }}) + .boop(new Func1() { + @Override + public Jail call(Object cylon) { + return new Jail(cylon); + }}) + .DESTROY() + .x(new ConvertToObservable()) + .reduce("Cylon Detector finished. Report:\n", new Func2() { + @Override + public String call(String a, String n) { + return a + n + "\n"; + }}) + .subscribe(subscriber); + subscriber.assertNoErrors(); + subscriber.assertCompleted(); + } + + @Test + public void testConvertToConcurrentQueue() { + final AtomicReference thrown = new AtomicReference(null); + final AtomicBoolean isFinished = new AtomicBoolean(false); + ConcurrentLinkedQueue queue = Observable.range(0,5) + .flatMap(new Func1>(){ + @Override + public Observable call(final Integer i) { + return Observable.range(0, 5) + .observeOn(Schedulers.io()) + .map(new Func1(){ + @Override + public Integer call(Integer k) { + try { + Thread.sleep(System.currentTimeMillis() % 100); + } catch (InterruptedException e) { + e.printStackTrace(); + } + return i + k; + }}); + }}) + .x(new Func1, ConcurrentLinkedQueue>() { + @Override + public ConcurrentLinkedQueue call(OnSubscribe onSubscribe) { + final ConcurrentLinkedQueue q = new ConcurrentLinkedQueue(); + onSubscribe.call(new Subscriber(){ + @Override + public void onCompleted() { + isFinished.set(true); + } + + @Override + public void onError(Throwable e) { + thrown.set(e); + } + + @Override + public void onNext(Integer t) { + q.add(t); + }}); + return q; + }}); + + int x = 0; + while(!isFinished.get()) { + Integer i = queue.poll(); + if (i != null) { + x++; + System.out.println(x + " item: " + i); + } + } + assertEquals(null, thrown.get()); + } +} diff --git a/src/test/java/rx/ObservableTests.java b/src/test/java/rx/ObservableTests.java index 5f1667deb6..55e43896d3 100644 --- a/src/test/java/rx/ObservableTests.java +++ b/src/test/java/rx/ObservableTests.java @@ -53,7 +53,6 @@ import rx.functions.Func0; import rx.functions.Func1; import rx.functions.Func2; -import rx.functions.Functions; import rx.observables.ConnectableObservable; import rx.observers.TestSubscriber; import rx.schedulers.TestScheduler; @@ -1157,4 +1156,19 @@ public void testForEachWithNull() { // .forEach(null); } + + @Test + public void testExtend() { + final TestSubscriber subscriber = new TestSubscriber(); + final Object value = new Object(); + Observable.just(value).x(new Func1,Object>(){ + @Override + public Object call(OnSubscribe onSubscribe) { + onSubscribe.call(subscriber); + subscriber.assertNoErrors(); + subscriber.assertCompleted(); + subscriber.assertValue(value); + return subscriber.getOnNextEvents().get(0); + }}); + } } From 662c9546ea4e687d7a8b7989d47fe4343c09a479 Mon Sep 17 00:00:00 2001 From: David Gross Date: Fri, 7 Aug 2015 15:59:03 -0700 Subject: [PATCH 28/81] eliminate javadoc compiler warnings, add "since" stub --- src/main/java/rx/Observable.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/main/java/rx/Observable.java b/src/main/java/rx/Observable.java index 246e54f023..79825d622b 100644 --- a/src/main/java/rx/Observable.java +++ b/src/main/java/rx/Observable.java @@ -110,12 +110,13 @@ public interface Operator extends Func1, Subscriber< } /** - * Passes all emitted values from {@code this} Observable to the provided {@link ConversionFunc} to be - * collected and returned as a single value. Note that it is legal for a {@link ConversionFunc} to - * return an Observable (enabling chaining). + * Passes all emitted values from this Observable to the provided conversion function to be collected and + * returned as a single value. Note that it is legal for a conversion function to return an Observable + * (enabling chaining). * * @param conversion a function that converts from this {@code Observable} to an {@code R} - * @return an instance of R created by the provided Conversion + * @return an instance of R created by the provided conversion function + * @since (if this graduates from Experimental/Beta to supported, replace this parenthetical with the release number) */ @Experimental public R x(Func1, ? extends R> conversion) { From 62871053b8eb75c4e3b757f5e35ff307a2843b5f Mon Sep 17 00:00:00 2001 From: akarnokd Date: Sun, 9 Aug 2015 11:25:00 +0200 Subject: [PATCH 29/81] Correct scheduler memory leak test for from(Executor) and added check for periodic tasks retention as well. --- .../schedulers/CachedThreadSchedulerTest.java | 63 +++---------- .../schedulers/ComputationSchedulerTests.java | 17 ++++ .../rx/schedulers/ExecutorSchedulerTest.java | 89 +++++++++++++++++-- 3 files changed, 110 insertions(+), 59 deletions(-) diff --git a/src/test/java/rx/schedulers/CachedThreadSchedulerTest.java b/src/test/java/rx/schedulers/CachedThreadSchedulerTest.java index 2b22e78068..9abb52b7ec 100644 --- a/src/test/java/rx/schedulers/CachedThreadSchedulerTest.java +++ b/src/test/java/rx/schedulers/CachedThreadSchedulerTest.java @@ -16,18 +16,13 @@ package rx.schedulers; -import java.lang.management.*; -import java.util.concurrent.*; - -import junit.framework.Assert; +import static org.junit.Assert.assertTrue; import org.junit.Test; -import rx.Observable; -import rx.Scheduler; +import rx.*; +import rx.Scheduler.Worker; import rx.functions.*; -import rx.internal.schedulers.NewThreadWorker; -import static org.junit.Assert.assertTrue; public class CachedThreadSchedulerTest extends AbstractSchedulerConcurrencyTests { @@ -74,49 +69,17 @@ public final void testHandledErrorIsNotDeliveredToThreadHandler() throws Interru @Test(timeout = 30000) public void testCancelledTaskRetention() throws InterruptedException { - System.out.println("Wait before GC"); - Thread.sleep(1000); - - System.out.println("GC"); - System.gc(); - - Thread.sleep(1000); - - - MemoryMXBean memoryMXBean = ManagementFactory.getMemoryMXBean(); - MemoryUsage memHeap = memoryMXBean.getHeapMemoryUsage(); - long initial = memHeap.getUsed(); - - System.out.printf("Starting: %.3f MB%n", initial / 1024.0 / 1024.0); - - Scheduler.Worker w = Schedulers.io().createWorker(); - for (int i = 0; i < 750000; i++) { - if (i % 50000 == 0) { - System.out.println(" -> still scheduling: " + i); - } - w.schedule(Actions.empty(), 1, TimeUnit.DAYS); + Worker w = Schedulers.io().createWorker(); + try { + ExecutorSchedulerTest.testCancelledRetention(w, false); + } finally { + w.unsubscribe(); } - - memHeap = memoryMXBean.getHeapMemoryUsage(); - long after = memHeap.getUsed(); - System.out.printf("Peak: %.3f MB%n", after / 1024.0 / 1024.0); - - w.unsubscribe(); - - System.out.println("Wait before second GC"); - Thread.sleep(NewThreadWorker.PURGE_FREQUENCY + 2000); - - System.out.println("Second GC"); - System.gc(); - - Thread.sleep(1000); - - memHeap = memoryMXBean.getHeapMemoryUsage(); - long finish = memHeap.getUsed(); - System.out.printf("After: %.3f MB%n", finish / 1024.0 / 1024.0); - - if (finish > initial * 5) { - Assert.fail(String.format("Tasks retained: %.3f -> %.3f -> %.3f", initial / 1024 / 1024.0, after / 1024 / 1024.0, finish / 1024 / 1024d)); + w = Schedulers.io().createWorker(); + try { + ExecutorSchedulerTest.testCancelledRetention(w, true); + } finally { + w.unsubscribe(); } } diff --git a/src/test/java/rx/schedulers/ComputationSchedulerTests.java b/src/test/java/rx/schedulers/ComputationSchedulerTests.java index 881224cfac..7191f60015 100644 --- a/src/test/java/rx/schedulers/ComputationSchedulerTests.java +++ b/src/test/java/rx/schedulers/ComputationSchedulerTests.java @@ -26,6 +26,7 @@ import rx.Observable; import rx.Scheduler; +import rx.Scheduler.Worker; import rx.functions.Action0; import rx.functions.Action1; import rx.functions.Func1; @@ -151,4 +152,20 @@ public final void testUnhandledErrorIsDeliveredToThreadHandler() throws Interrup public final void testHandledErrorIsNotDeliveredToThreadHandler() throws InterruptedException { SchedulerTests.testHandledErrorIsNotDeliveredToThreadHandler(getScheduler()); } + + @Test(timeout = 30000) + public void testCancelledTaskRetention() throws InterruptedException { + Worker w = Schedulers.computation().createWorker(); + try { + ExecutorSchedulerTest.testCancelledRetention(w, false); + } finally { + w.unsubscribe(); + } + w = Schedulers.computation().createWorker(); + try { + ExecutorSchedulerTest.testCancelledRetention(w, true); + } finally { + w.unsubscribe(); + } + } } diff --git a/src/test/java/rx/schedulers/ExecutorSchedulerTest.java b/src/test/java/rx/schedulers/ExecutorSchedulerTest.java index cdefabc757..ed4e03213d 100644 --- a/src/test/java/rx/schedulers/ExecutorSchedulerTest.java +++ b/src/test/java/rx/schedulers/ExecutorSchedulerTest.java @@ -48,8 +48,8 @@ public final void testUnhandledErrorIsDeliveredToThreadHandler() throws Interrup public final void testHandledErrorIsNotDeliveredToThreadHandler() throws InterruptedException { SchedulerTests.testHandledErrorIsNotDeliveredToThreadHandler(getScheduler()); } - @Test(timeout = 30000) - public void testCancelledTaskRetention() throws InterruptedException { + + public static void testCancelledRetention(Scheduler.Worker w, boolean periodic) throws InterruptedException { System.out.println("Wait before GC"); Thread.sleep(1000); @@ -64,13 +64,32 @@ public void testCancelledTaskRetention() throws InterruptedException { long initial = memHeap.getUsed(); System.out.printf("Starting: %.3f MB%n", initial / 1024.0 / 1024.0); - - Scheduler.Worker w = Schedulers.io().createWorker(); - for (int i = 0; i < 500000; i++) { - if (i % 50000 == 0) { - System.out.println(" -> still scheduling: " + i); + + int n = 500 * 1000; + if (periodic) { + final CountDownLatch cdl = new CountDownLatch(n); + final Action0 action = new Action0() { + @Override + public void call() { + cdl.countDown(); + } + }; + for (int i = 0; i < n; i++) { + if (i % 50000 == 0) { + System.out.println(" -> still scheduling: " + i); + } + w.schedulePeriodically(action, 0, 1, TimeUnit.DAYS); + } + + System.out.println("Waiting for the first round to finish..."); + cdl.await(); + } else { + for (int i = 0; i < n; i++) { + if (i % 50000 == 0) { + System.out.println(" -> still scheduling: " + i); + } + w.schedule(Actions.empty(), 1, TimeUnit.DAYS); } - w.schedule(Actions.empty(), 1, TimeUnit.DAYS); } memHeap = memoryMXBean.getHeapMemoryUsage(); @@ -95,7 +114,30 @@ public void testCancelledTaskRetention() throws InterruptedException { fail(String.format("Tasks retained: %.3f -> %.3f -> %.3f", initial / 1024 / 1024.0, after / 1024 / 1024.0, finish / 1024 / 1024d)); } } - + + @Test(timeout = 30000) + public void testCancelledTaskRetention() throws InterruptedException { + ExecutorService exec = Executors.newSingleThreadExecutor(); + Scheduler s = Schedulers.from(exec); + try { + Scheduler.Worker w = s.createWorker(); + try { + testCancelledRetention(w, false); + } finally { + w.unsubscribe(); + } + + w = s.createWorker(); + try { + testCancelledRetention(w, true); + } finally { + w.unsubscribe(); + } + } finally { + exec.shutdownNow(); + } + } + /** A simple executor which queues tasks and executes them one-by-one if executeOne() is called. */ static final class TestExecutor implements Executor { final ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue(); @@ -204,4 +246,33 @@ public void execute(Runnable command) { assertFalse(w.tasks.hasSubscriptions()); } + + @Test + public void testNoPeriodicTimedTaskPartRetention() throws InterruptedException { + Executor e = new Executor() { + @Override + public void execute(Runnable command) { + command.run(); + } + }; + ExecutorSchedulerWorker w = (ExecutorSchedulerWorker)Schedulers.from(e).createWorker(); + + final CountDownLatch cdl = new CountDownLatch(1); + final Action0 action = new Action0() { + @Override + public void call() { + cdl.countDown(); + } + }; + + Subscription s = w.schedulePeriodically(action, 0, 1, TimeUnit.DAYS); + + assertTrue(w.tasks.hasSubscriptions()); + + cdl.await(); + + s.unsubscribe(); + + assertFalse(w.tasks.hasSubscriptions()); + } } From 829c6ce35412ae420e3acb652d63f6242c51892e Mon Sep 17 00:00:00 2001 From: Artem Zinnatullin Date: Mon, 10 Aug 2015 19:18:03 +0300 Subject: [PATCH 30/81] Fix for BackpressureUtils method javadoc --- src/main/java/rx/internal/operators/BackpressureUtils.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/main/java/rx/internal/operators/BackpressureUtils.java b/src/main/java/rx/internal/operators/BackpressureUtils.java index c62eefcbbc..505b248553 100644 --- a/src/main/java/rx/internal/operators/BackpressureUtils.java +++ b/src/main/java/rx/internal/operators/BackpressureUtils.java @@ -61,9 +61,7 @@ public static long getAndAddRequest(AtomicLongFieldUpdater requested, T o * {@code requested} field to {@code Long.MAX_VALUE}. * * @param requested - * atomic field updater for a request count - * @param object - * contains the field updated by the updater + * atomic long that should be updated * @param n * the number of requests to add to the requested count * @return requested value just prior to successful addition From 893b417b3af77d107860ca416fcb876aef627fd7 Mon Sep 17 00:00:00 2001 From: Artem Zinnatullin Date: Mon, 10 Aug 2015 19:51:03 +0300 Subject: [PATCH 31/81] Remove redundant cast in Exceptions --- src/main/java/rx/exceptions/Exceptions.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/rx/exceptions/Exceptions.java b/src/main/java/rx/exceptions/Exceptions.java index 8ac7091def..b8907bf436 100644 --- a/src/main/java/rx/exceptions/Exceptions.java +++ b/src/main/java/rx/exceptions/Exceptions.java @@ -77,7 +77,7 @@ public static void throwIfFatal(Throwable t) { if (t instanceof OnErrorNotImplementedException) { throw (OnErrorNotImplementedException) t; } else if (t instanceof OnErrorFailedException) { - Throwable cause = ((OnErrorFailedException) t).getCause(); + Throwable cause = t.getCause(); if (cause instanceof RuntimeException) { throw (RuntimeException) cause; } else { From df9d3cbde6c437b8c13b093a3b835c50dc2694ca Mon Sep 17 00:00:00 2001 From: Artem Zinnatullin Date: Tue, 11 Aug 2015 02:15:48 +0300 Subject: [PATCH 32/81] Remove unnecessary static modifier --- src/main/java/rx/Notification.java | 2 +- src/main/java/rx/Single.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/java/rx/Notification.java b/src/main/java/rx/Notification.java index 95d557726a..17a23d1031 100644 --- a/src/main/java/rx/Notification.java +++ b/src/main/java/rx/Notification.java @@ -162,7 +162,7 @@ public void accept(Observer observer) { } } - public static enum Kind { + public enum Kind { OnNext, OnError, OnCompleted } diff --git a/src/main/java/rx/Single.java b/src/main/java/rx/Single.java index ede27c8eb6..7fbf369b79 100644 --- a/src/main/java/rx/Single.java +++ b/src/main/java/rx/Single.java @@ -147,7 +147,7 @@ public final static Single create(OnSubscribe f) { /** * Invoked when Single.execute is called. */ - public static interface OnSubscribe extends Action1> { + public interface OnSubscribe extends Action1> { // cover for generics insanity } @@ -235,7 +235,7 @@ public Single compose(Transformer transformer) { * * @warn more complete description needed */ - public static interface Transformer extends Func1, Single> { + public interface Transformer extends Func1, Single> { // cover for generics insanity } From e31c0d3ef64360d648f532dbe179690e4edfec76 Mon Sep 17 00:00:00 2001 From: Dave Moten Date: Fri, 24 Jul 2015 15:13:19 +1000 Subject: [PATCH 33/81] fix SynchronizedQueue.equals --- .../rx/internal/util/SynchronizedQueue.java | 20 +++++++++++++++---- .../internal/util/SynchronizedQueueTest.java | 15 ++++++++++++++ 2 files changed, 31 insertions(+), 4 deletions(-) create mode 100644 src/test/java/rx/internal/util/SynchronizedQueueTest.java diff --git a/src/main/java/rx/internal/util/SynchronizedQueue.java b/src/main/java/rx/internal/util/SynchronizedQueue.java index 9fe867d93a..8f0c4a7372 100644 --- a/src/main/java/rx/internal/util/SynchronizedQueue.java +++ b/src/main/java/rx/internal/util/SynchronizedQueue.java @@ -99,13 +99,25 @@ public synchronized String toString() { } @Override - public synchronized boolean equals(Object o) { - return list.equals(o); + public int hashCode() { + return list.hashCode(); } @Override - public synchronized int hashCode() { - return list.hashCode(); + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + SynchronizedQueue other = (SynchronizedQueue) obj; + if (list == null) { + if (other.list != null) + return false; + } else if (!list.equals(other.list)) + return false; + return true; } @Override diff --git a/src/test/java/rx/internal/util/SynchronizedQueueTest.java b/src/test/java/rx/internal/util/SynchronizedQueueTest.java new file mode 100644 index 0000000000..98779ea1b9 --- /dev/null +++ b/src/test/java/rx/internal/util/SynchronizedQueueTest.java @@ -0,0 +1,15 @@ +package rx.internal.util; + +import static org.junit.Assert.assertTrue; + +import org.junit.Test; + +public class SynchronizedQueueTest { + + @Test + public void testEquals() { + SynchronizedQueue q = new SynchronizedQueue(); + assertTrue(q.equals(q)); + } + +} From 7980405874332df8d2e94a412d8fab757fd4303d Mon Sep 17 00:00:00 2001 From: Jacek Rzeniewicz Date: Thu, 30 Jul 2015 00:29:16 +0100 Subject: [PATCH 34/81] Remove redundant type parameter in EmptyAction --- src/main/java/rx/functions/Actions.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/rx/functions/Actions.java b/src/main/java/rx/functions/Actions.java index 2002995487..342cfd030c 100644 --- a/src/main/java/rx/functions/Actions.java +++ b/src/main/java/rx/functions/Actions.java @@ -24,14 +24,14 @@ private Actions() { } @SuppressWarnings("unchecked") - public static final EmptyAction empty() { + public static final EmptyAction empty() { return EMPTY_ACTION; } @SuppressWarnings("rawtypes") private static final EmptyAction EMPTY_ACTION = new EmptyAction(); - private static final class EmptyAction implements + private static final class EmptyAction implements Action0, Action1, Action2, From 9004776f02538862ebba39c0e37ccd658d7451f4 Mon Sep 17 00:00:00 2001 From: akarnokd Date: Mon, 27 Jul 2015 10:16:26 +0200 Subject: [PATCH 35/81] Test coverage for the observers package. --- .../java/rx/observers/SerializedObserver.java | 188 +++----- src/main/java/rx/observers/TestObserver.java | 14 +- .../java/rx/observers/TestSubscriber.java | 13 +- src/test/java/rx/observers/ObserversTest.java | 189 ++++++++ .../java/rx/observers/SafeObserverTest.java | 51 +- .../java/rx/observers/SafeSubscriberTest.java | 230 +++++++++ .../rx/observers/SerializedObserverTest.java | 199 ++++++-- .../java/rx/observers/SubscribersTest.java | 188 ++++++++ .../java/rx/observers/TestObserverTest.java | 117 ++++- .../java/rx/observers/TestSubscriberTest.java | 455 +++++++++++++++++- 10 files changed, 1475 insertions(+), 169 deletions(-) create mode 100644 src/test/java/rx/observers/ObserversTest.java create mode 100644 src/test/java/rx/observers/SafeSubscriberTest.java create mode 100644 src/test/java/rx/observers/SubscribersTest.java diff --git a/src/main/java/rx/observers/SerializedObserver.java b/src/main/java/rx/observers/SerializedObserver.java index 86ca42f8cf..8125ce54e6 100644 --- a/src/main/java/rx/observers/SerializedObserver.java +++ b/src/main/java/rx/observers/SerializedObserver.java @@ -16,7 +16,8 @@ package rx.observers; import rx.Observer; -import rx.exceptions.Exceptions; +import rx.exceptions.*; +import rx.internal.operators.NotificationLite; /** * Enforces single-threaded, serialized, ordered execution of {@link #onNext}, {@link #onCompleted}, and @@ -35,13 +36,15 @@ public class SerializedObserver implements Observer { private final Observer actual; - private boolean emitting = false; - private boolean terminated = false; + private boolean emitting; + /** Set to true if a terminal event was received. */ + private volatile boolean terminated; + /** If not null, it indicates more work. */ private FastList queue; + private final NotificationLite nl = NotificationLite.instance(); - private static final int MAX_DRAIN_ITERATION = Integer.MAX_VALUE; - private static final Object NULL_SENTINEL = new Object(); - private static final Object COMPLETE_SENTINEL = new Object(); + /** Number of iterations without additional safepoint poll in the drain loop. */ + private static final int MAX_DRAIN_ITERATION = 1024; static final class FastList { Object[] array; @@ -64,150 +67,119 @@ public void add(Object o) { } } - private static final class ErrorSentinel { - final Throwable e; - - ErrorSentinel(Throwable e) { - this.e = e; - } - } - public SerializedObserver(Observer s) { this.actual = s; } @Override - public void onCompleted() { - FastList list; + public void onNext(T t) { + if (terminated) { + return; + } synchronized (this) { if (terminated) { return; } - terminated = true; if (emitting) { - if (queue == null) { - queue = new FastList(); + FastList list = queue; + if (list == null) { + list = new FastList(); + queue = list; } - queue.add(COMPLETE_SENTINEL); + list.add(nl.next(t)); return; } emitting = true; - list = queue; - queue = null; } - drainQueue(list); - actual.onCompleted(); + try { + actual.onNext(t); + } catch (Throwable e) { + terminated = true; + Exceptions.throwIfFatal(e); + actual.onError(OnErrorThrowable.addValueAsLastCause(e, t)); + return; + } + for (;;) { + for (int i = 0; i < MAX_DRAIN_ITERATION; i++) { + FastList list; + synchronized (this) { + list = queue; + if (list == null) { + emitting = false; + return; + } + queue = null; + } + for (Object o : list.array) { + if (o == null) { + break; + } + try { + if (nl.accept(actual, o)) { + terminated = true; + return; + } + } catch (Throwable e) { + terminated = true; + Exceptions.throwIfFatal(e); + actual.onError(OnErrorThrowable.addValueAsLastCause(e, t)); + return; + } + } + } + } } - + @Override public void onError(final Throwable e) { Exceptions.throwIfFatal(e); - FastList list; + if (terminated) { + return; + } synchronized (this) { if (terminated) { return; } + terminated = true; if (emitting) { - if (queue == null) { - queue = new FastList(); + /* + * FIXME: generally, errors jump the queue but this wasn't true + * for SerializedObserver and may break existing expectations. + */ + FastList list = queue; + if (list == null) { + list = new FastList(); + queue = list; } - queue.add(new ErrorSentinel(e)); + list.add(nl.error(e)); return; } emitting = true; - list = queue; - queue = null; } - drainQueue(list); actual.onError(e); - synchronized(this) { - emitting = false; - } } @Override - public void onNext(T t) { - FastList list; - + public void onCompleted() { + if (terminated) { + return; + } synchronized (this) { if (terminated) { return; } + terminated = true; if (emitting) { - if (queue == null) { - queue = new FastList(); + FastList list = queue; + if (list == null) { + list = new FastList(); + queue = list; } - queue.add(t != null ? t : NULL_SENTINEL); - // another thread is emitting so we add to the queue and return + list.add(nl.completed()); return; } - // we can emit emitting = true; - // reference to the list to drain before emitting our value - list = queue; - queue = null; - } - - // we only get here if we won the right to emit, otherwise we returned in the if(emitting) block above - boolean skipFinal = false; - try { - int iter = MAX_DRAIN_ITERATION; - do { - drainQueue(list); - if (iter == MAX_DRAIN_ITERATION) { - // after the first draining we emit our own value - actual.onNext(t); - } - --iter; - if (iter > 0) { - synchronized (this) { - list = queue; - queue = null; - if (list == null) { - emitting = false; - skipFinal = true; - return; - } - } - } - } while (iter > 0); - } finally { - if (!skipFinal) { - synchronized (this) { - if (terminated) { - list = queue; - queue = null; - } else { - emitting = false; - list = null; - } - } - } - } - - // this will only drain if terminated (done here outside of synchronized block) - drainQueue(list); - } - - void drainQueue(FastList list) { - if (list == null || list.size == 0) { - return; - } - for (Object v : list.array) { - if (v == null) { - break; - } - if (v == NULL_SENTINEL) { - actual.onNext(null); - } else if (v == COMPLETE_SENTINEL) { - actual.onCompleted(); - } else if (v.getClass() == ErrorSentinel.class) { - actual.onError(((ErrorSentinel) v).e); - } else { - @SuppressWarnings("unchecked") - T t = (T)v; - actual.onNext(t); - } } + actual.onCompleted(); } } diff --git a/src/main/java/rx/observers/TestObserver.java b/src/main/java/rx/observers/TestObserver.java index e7f02131b6..c20784187a 100644 --- a/src/main/java/rx/observers/TestObserver.java +++ b/src/main/java/rx/observers/TestObserver.java @@ -117,13 +117,17 @@ public void assertReceivedOnNext(List items) { } for (int i = 0; i < items.size(); i++) { - if (items.get(i) == null) { + T expected = items.get(i); + T actual = onNextEvents.get(i); + if (expected == null) { // check for null equality - if (onNextEvents.get(i) != null) { - throw new AssertionError("Value at index: " + i + " expected to be [null] but was: [" + onNextEvents.get(i) + "]"); + if (actual != null) { + throw new AssertionError("Value at index: " + i + " expected to be [null] but was: [" + actual + "]"); } - } else if (!items.get(i).equals(onNextEvents.get(i))) { - throw new AssertionError("Value at index: " + i + " expected to be [" + items.get(i) + "] (" + items.get(i).getClass().getSimpleName() + ") but was: [" + onNextEvents.get(i) + "] (" + onNextEvents.get(i).getClass().getSimpleName() + ")"); + } else if (!expected.equals(actual)) { + throw new AssertionError("Value at index: " + i + + " expected to be [" + expected + "] (" + expected.getClass().getSimpleName() + + ") but was: [" + actual + "] (" + (actual != null ? actual.getClass().getSimpleName() : "null") + ")"); } } diff --git a/src/main/java/rx/observers/TestSubscriber.java b/src/main/java/rx/observers/TestSubscriber.java index a2255cf401..2d46a25179 100644 --- a/src/main/java/rx/observers/TestSubscriber.java +++ b/src/main/java/rx/observers/TestSubscriber.java @@ -258,10 +258,15 @@ public void assertUnsubscribed() { * if this {@code Subscriber} has received one or more {@code onError} notifications */ public void assertNoErrors() { - if (getOnErrorEvents().size() > 0) { - // can't use AssertionError because (message, cause) doesn't exist until Java 7 - throw new RuntimeException("Unexpected onError events: " + getOnErrorEvents().size(), getOnErrorEvents().get(0)); - // TODO possibly check for Java7+ and then use AssertionError at runtime (since we always compile with 7) + List onErrorEvents = getOnErrorEvents(); + if (onErrorEvents.size() > 0) { + AssertionError ae = new AssertionError("Unexpected onError events: " + getOnErrorEvents().size()); + if (onErrorEvents.size() == 1) { + ae.initCause(getOnErrorEvents().get(0)); + } else { + ae.initCause(new CompositeException(onErrorEvents)); + } + throw ae; } } diff --git a/src/test/java/rx/observers/ObserversTest.java b/src/test/java/rx/observers/ObserversTest.java new file mode 100644 index 0000000000..df8b3aae99 --- /dev/null +++ b/src/test/java/rx/observers/ObserversTest.java @@ -0,0 +1,189 @@ +/** + * Copyright 2014 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.observers; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +import java.lang.reflect.*; +import java.util.concurrent.atomic.*; + +import org.junit.Test; + +import rx.exceptions.*; +import rx.functions.*; + +public class ObserversTest { + @Test + public void testNotInstantiable() { + try { + Constructor c = Observers.class.getDeclaredConstructor(); + c.setAccessible(true); + Object instance = c.newInstance(); + fail("Could instantiate Actions! " + instance); + } catch (NoSuchMethodException ex) { + ex.printStackTrace(); + } catch (InvocationTargetException ex) { + ex.printStackTrace(); + } catch (InstantiationException ex) { + ex.printStackTrace(); + } catch (IllegalAccessException ex) { + ex.printStackTrace(); + } + } + + @Test + public void testEmptyOnErrorNotImplemented() { + try { + Observers.empty().onError(new TestException()); + fail("OnErrorNotImplementedException not thrown!"); + } catch (OnErrorNotImplementedException ex) { + if (!(ex.getCause() instanceof TestException)) { + fail("TestException not wrapped, instead: " + ex.getCause()); + } + } + } + @Test + public void testCreate1OnErrorNotImplemented() { + try { + Observers.create(Actions.empty()).onError(new TestException()); + fail("OnErrorNotImplementedException not thrown!"); + } catch (OnErrorNotImplementedException ex) { + if (!(ex.getCause() instanceof TestException)) { + fail("TestException not wrapped, instead: " + ex.getCause()); + } + } + } + @Test(expected = IllegalArgumentException.class) + public void testCreate1Null() { + Observers.create(null); + } + @Test(expected = IllegalArgumentException.class) + public void testCreate2Null() { + Action1 throwAction = Actions.empty(); + Observers.create(null, throwAction); + } + @Test(expected = IllegalArgumentException.class) + public void testCreate3Null() { + Observers.create(Actions.empty(), null); + } + + @Test(expected = IllegalArgumentException.class) + public void testCreate4Null() { + Action1 throwAction = Actions.empty(); + Observers.create(null, throwAction, Actions.empty()); + } + @Test(expected = IllegalArgumentException.class) + public void testCreate5Null() { + Observers.create(Actions.empty(), null, Actions.empty()); + } + @Test(expected = IllegalArgumentException.class) + public void testCreate6Null() { + Action1 throwAction = Actions.empty(); + Observers.create(Actions.empty(), throwAction, null); + } + + @Test + public void testCreate1Value() { + final AtomicInteger value = new AtomicInteger(); + Action1 action = new Action1() { + @Override + public void call(Integer t) { + value.set(t); + } + }; + Observers.create(action).onNext(1); + + assertEquals(1, value.get()); + } + @Test + public void testCreate2Value() { + final AtomicInteger value = new AtomicInteger(); + Action1 action = new Action1() { + @Override + public void call(Integer t) { + value.set(t); + } + }; + Action1 throwAction = Actions.empty(); + Observers.create(action, throwAction).onNext(1); + + assertEquals(1, value.get()); + } + + @Test + public void testCreate3Value() { + final AtomicInteger value = new AtomicInteger(); + Action1 action = new Action1() { + @Override + public void call(Integer t) { + value.set(t); + } + }; + Action1 throwAction = Actions.empty(); + Observers.create(action, throwAction, Actions.empty()).onNext(1); + + assertEquals(1, value.get()); + } + + @Test + public void testError2() { + final AtomicReference value = new AtomicReference(); + Action1 action = new Action1() { + @Override + public void call(Throwable t) { + value.set(t); + } + }; + TestException exception = new TestException(); + Observers.create(Actions.empty(), action).onError(exception); + + assertEquals(exception, value.get()); + } + + @Test + public void testError3() { + final AtomicReference value = new AtomicReference(); + Action1 action = new Action1() { + @Override + public void call(Throwable t) { + value.set(t); + } + }; + TestException exception = new TestException(); + Observers.create(Actions.empty(), action, Actions.empty()).onError(exception); + + assertEquals(exception, value.get()); + } + + @Test + public void testCompleted() { + Action0 action = mock(Action0.class); + + Action1 throwAction = Actions.empty(); + Observers.create(Actions.empty(), throwAction, action).onCompleted(); + + verify(action).call(); + } + + @Test + public void testEmptyCompleted() { + Observers.create(Actions.empty()).onCompleted(); + + Action1 throwAction = Actions.empty(); + Observers.create(Actions.empty(), throwAction).onCompleted(); + } +} diff --git a/src/test/java/rx/observers/SafeObserverTest.java b/src/test/java/rx/observers/SafeObserverTest.java index 584c6ee117..1083e995c7 100644 --- a/src/test/java/rx/observers/SafeObserverTest.java +++ b/src/test/java/rx/observers/SafeObserverTest.java @@ -15,11 +15,7 @@ */ package rx.observers; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.Assert.*; import java.util.List; import java.util.concurrent.atomic.AtomicReference; @@ -27,9 +23,7 @@ import org.junit.Test; import rx.Subscriber; -import rx.exceptions.CompositeException; -import rx.exceptions.OnErrorFailedException; -import rx.exceptions.OnErrorNotImplementedException; +import rx.exceptions.*; import rx.functions.Action0; import rx.subscriptions.Subscriptions; @@ -462,4 +456,45 @@ public SafeObserverTestException(String message) { super(message); } } + + @Test + public void testOnCompletedThrows() { + final AtomicReference error = new AtomicReference(); + SafeSubscriber s = new SafeSubscriber(new Subscriber() { + @Override + public void onNext(Integer t) { + + } + @Override + public void onError(Throwable e) { + error.set(e); + } + @Override + public void onCompleted() { + throw new TestException(); + } + }); + + s.onCompleted(); + + assertTrue("Error not received", error.get() instanceof TestException); + } + + @Test + public void testActual() { + Subscriber actual = new Subscriber() { + @Override + public void onNext(Integer t) { + } + @Override + public void onError(Throwable e) { + } + @Override + public void onCompleted() { + } + }; + SafeSubscriber s = new SafeSubscriber(actual); + + assertSame(actual, s.getActual()); + } } diff --git a/src/test/java/rx/observers/SafeSubscriberTest.java b/src/test/java/rx/observers/SafeSubscriberTest.java new file mode 100644 index 0000000000..85c2d7b07f --- /dev/null +++ b/src/test/java/rx/observers/SafeSubscriberTest.java @@ -0,0 +1,230 @@ +/** + * Copyright 2014 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.observers; + +import static org.junit.Assert.assertTrue; + +import java.lang.reflect.Method; + +import org.junit.*; + +import rx.exceptions.*; +import rx.functions.Action0; +import rx.plugins.*; +import rx.subscriptions.Subscriptions; + +public class SafeSubscriberTest { + + @Before + @After + public void resetBefore() { + RxJavaPlugins ps = RxJavaPlugins.getInstance(); + + try { + Method m = ps.getClass().getDeclaredMethod("reset"); + m.setAccessible(true); + m.invoke(ps); + } catch (Throwable ex) { + ex.printStackTrace(); + } + } + + @Test + public void testOnCompletedThrows() { + TestSubscriber ts = new TestSubscriber() { + @Override + public void onCompleted() { + throw new TestException(); + } + }; + SafeSubscriber safe = new SafeSubscriber(ts); + + safe.onCompleted(); + + assertTrue(safe.isUnsubscribed()); + } + + @Test + public void testOnCompletedThrows2() { + TestSubscriber ts = new TestSubscriber() { + @Override + public void onCompleted() { + throw new OnErrorNotImplementedException(new TestException()); + } + }; + SafeSubscriber safe = new SafeSubscriber(ts); + + try { + safe.onCompleted(); + } catch (OnErrorNotImplementedException ex) { + // expected + } + + assertTrue(safe.isUnsubscribed()); + } + + @Test + public void testPluginException() { + RxJavaPlugins.getInstance().registerErrorHandler(new RxJavaErrorHandler() { + @Override + public void handleError(Throwable e) { + throw new RuntimeException(); + } + }); + + TestSubscriber ts = new TestSubscriber() { + @Override + public void onCompleted() { + throw new TestException(); + } + }; + SafeSubscriber safe = new SafeSubscriber(ts); + + safe.onCompleted(); + } + + @Test(expected = OnErrorFailedException.class) + public void testPluginExceptionWhileOnErrorUnsubscribeThrows() { + RxJavaPlugins.getInstance().registerErrorHandler(new RxJavaErrorHandler() { + int calls; + @Override + public void handleError(Throwable e) { + if (++calls == 2) { + throw new RuntimeException(); + } + } + }); + + TestSubscriber ts = new TestSubscriber(); + SafeSubscriber safe = new SafeSubscriber(ts); + safe.add(Subscriptions.create(new Action0() { + @Override + public void call() { + throw new RuntimeException(); + } + })); + + safe.onError(new TestException()); + } + + @Test(expected = RuntimeException.class) + public void testPluginExceptionWhileOnErrorThrowsNotImplAndUnsubscribeThrows() { + RxJavaPlugins.getInstance().registerErrorHandler(new RxJavaErrorHandler() { + int calls; + @Override + public void handleError(Throwable e) { + if (++calls == 2) { + throw new RuntimeException(); + } + } + }); + + TestSubscriber ts = new TestSubscriber() { + @Override + public void onError(Throwable e) { + throw new OnErrorNotImplementedException(e); + } + }; + SafeSubscriber safe = new SafeSubscriber(ts); + safe.add(Subscriptions.create(new Action0() { + @Override + public void call() { + throw new RuntimeException(); + } + })); + + safe.onError(new TestException()); + } + + @Test(expected = OnErrorFailedException.class) + public void testPluginExceptionWhileOnErrorThrows() { + RxJavaPlugins.getInstance().registerErrorHandler(new RxJavaErrorHandler() { + int calls; + @Override + public void handleError(Throwable e) { + if (++calls == 2) { + throw new RuntimeException(); + } + } + }); + + TestSubscriber ts = new TestSubscriber() { + @Override + public void onError(Throwable e) { + throw new RuntimeException(e); + } + }; + SafeSubscriber safe = new SafeSubscriber(ts); + + safe.onError(new TestException()); + } + @Test(expected = OnErrorFailedException.class) + public void testPluginExceptionWhileOnErrorThrowsAndUnsubscribeThrows() { + RxJavaPlugins.getInstance().registerErrorHandler(new RxJavaErrorHandler() { + int calls; + @Override + public void handleError(Throwable e) { + if (++calls == 2) { + throw new RuntimeException(); + } + } + }); + + TestSubscriber ts = new TestSubscriber() { + @Override + public void onError(Throwable e) { + throw new RuntimeException(e); + } + }; + SafeSubscriber safe = new SafeSubscriber(ts); + safe.add(Subscriptions.create(new Action0() { + @Override + public void call() { + throw new RuntimeException(); + } + })); + + safe.onError(new TestException()); + } + @Test(expected = OnErrorFailedException.class) + public void testPluginExceptionWhenUnsubscribing2() { + RxJavaPlugins.getInstance().registerErrorHandler(new RxJavaErrorHandler() { + int calls; + @Override + public void handleError(Throwable e) { + if (++calls == 3) { + throw new RuntimeException(); + } + } + }); + + TestSubscriber ts = new TestSubscriber() { + @Override + public void onError(Throwable e) { + throw new RuntimeException(e); + } + }; + SafeSubscriber safe = new SafeSubscriber(ts); + safe.add(Subscriptions.create(new Action0() { + @Override + public void call() { + throw new RuntimeException(); + } + })); + + safe.onError(new TestException()); + } +} diff --git a/src/test/java/rx/observers/SerializedObserverTest.java b/src/test/java/rx/observers/SerializedObserverTest.java index b469c131d4..a14f146e75 100644 --- a/src/test/java/rx/observers/SerializedObserverTest.java +++ b/src/test/java/rx/observers/SerializedObserverTest.java @@ -15,35 +15,20 @@ */ package rx.observers; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.Assert.*; import static org.mockito.Matchers.any; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; - -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; - -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Test; -import org.mockito.Mock; -import org.mockito.MockitoAnnotations; - -import rx.Observable; +import static org.mockito.Mockito.*; + +import java.util.Arrays; +import java.util.concurrent.*; +import java.util.concurrent.atomic.*; + +import org.junit.*; +import org.mockito.*; + +import rx.*; import rx.Observable.OnSubscribe; -import rx.Observer; -import rx.Subscriber; -import rx.Subscription; +import rx.exceptions.TestException; import rx.schedulers.Schedulers; public class SerializedObserverTest { @@ -813,4 +798,164 @@ protected void captureMaxThreads() { } } + + @Test + public void testSerializeNull() { + final AtomicReference> serial = new AtomicReference>(); + TestObserver to = new TestObserver() { + @Override + public void onNext(Integer t) { + if (t != null && t == 0) { + serial.get().onNext(null); + } + super.onNext(t); + } + }; + + SerializedObserver sobs = new SerializedObserver(to); + serial.set(sobs); + + sobs.onNext(0); + + to.assertReceivedOnNext(Arrays.asList(0, null)); + } + + @Test + public void testSerializeAllowsOnError() { + TestObserver to = new TestObserver() { + @Override + public void onNext(Integer t) { + throw new TestException(); + } + }; + + SerializedObserver sobs = new SerializedObserver(to); + + try { + sobs.onNext(0); + } catch (TestException ex) { + sobs.onError(ex); + } + + assertEquals(1, to.getOnErrorEvents().size()); + assertTrue(to.getOnErrorEvents().get(0) instanceof TestException); + } + + @Test + public void testSerializeReentrantNullAndComplete() { + final AtomicReference> serial = new AtomicReference>(); + TestObserver to = new TestObserver() { + @Override + public void onNext(Integer t) { + serial.get().onCompleted(); + throw new TestException(); + } + }; + + SerializedObserver sobs = new SerializedObserver(to); + serial.set(sobs); + + try { + sobs.onNext(0); + } catch (TestException ex) { + sobs.onError(ex); + } + + assertEquals(1, to.getOnErrorEvents().size()); + assertTrue(to.getOnErrorEvents().get(0) instanceof TestException); + assertTrue(to.getOnCompletedEvents().isEmpty()); + } + + @Test + public void testSerializeReentrantNullAndError() { + final AtomicReference> serial = new AtomicReference>(); + TestObserver to = new TestObserver() { + @Override + public void onNext(Integer t) { + serial.get().onError(new RuntimeException()); + throw new TestException(); + } + }; + + SerializedObserver sobs = new SerializedObserver(to); + serial.set(sobs); + + try { + sobs.onNext(0); + } catch (TestException ex) { + sobs.onError(ex); + } + + assertEquals(1, to.getOnErrorEvents().size()); + assertTrue(to.getOnErrorEvents().get(0) instanceof TestException); + assertTrue(to.getOnCompletedEvents().isEmpty()); + } + + @Test + public void testSerializeDrainPhaseThrows() { + final AtomicReference> serial = new AtomicReference>(); + TestObserver to = new TestObserver() { + @Override + public void onNext(Integer t) { + if (t != null && t == 0) { + serial.get().onNext(null); + } else + if (t == null) { + throw new TestException(); + } + super.onNext(t); + } + }; + + SerializedObserver sobs = new SerializedObserver(to); + serial.set(sobs); + + sobs.onNext(0); + + to.assertReceivedOnNext(Arrays.asList(0)); + assertEquals(1, to.getOnErrorEvents().size()); + assertTrue(to.getOnErrorEvents().get(0) instanceof TestException); + } + + @Test + public void testErrorReentry() { + final AtomicReference> serial = new AtomicReference>(); + + TestSubscriber ts = new TestSubscriber() { + @Override + public void onNext(Integer v) { + serial.get().onError(new TestException()); + serial.get().onError(new TestException()); + super.onNext(v); + } + }; + SerializedObserver sobs = new SerializedObserver(ts); + serial.set(sobs); + + sobs.onNext(1); + + ts.assertValue(1); + ts.assertError(TestException.class); + } + @Test + public void testCompleteReentry() { + final AtomicReference> serial = new AtomicReference>(); + + TestSubscriber ts = new TestSubscriber() { + @Override + public void onNext(Integer v) { + serial.get().onCompleted(); + serial.get().onCompleted(); + super.onNext(v); + } + }; + SerializedObserver sobs = new SerializedObserver(ts); + serial.set(sobs); + + sobs.onNext(1); + + ts.assertValue(1); + ts.assertCompleted(); + ts.assertNoErrors(); + } } diff --git a/src/test/java/rx/observers/SubscribersTest.java b/src/test/java/rx/observers/SubscribersTest.java new file mode 100644 index 0000000000..241ecae9af --- /dev/null +++ b/src/test/java/rx/observers/SubscribersTest.java @@ -0,0 +1,188 @@ +/** + * Copyright 2014 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.observers; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +import java.lang.reflect.*; +import java.util.concurrent.atomic.*; + +import org.junit.Test; + +import rx.exceptions.*; +import rx.functions.*; + +public class SubscribersTest { + @Test + public void testNotInstantiable() { + try { + Constructor c = Subscribers.class.getDeclaredConstructor(); + c.setAccessible(true); + Object instance = c.newInstance(); + fail("Could instantiate Actions! " + instance); + } catch (NoSuchMethodException ex) { + ex.printStackTrace(); + } catch (InvocationTargetException ex) { + ex.printStackTrace(); + } catch (InstantiationException ex) { + ex.printStackTrace(); + } catch (IllegalAccessException ex) { + ex.printStackTrace(); + } + } + + @Test + public void testEmptyOnErrorNotImplemented() { + try { + Subscribers.empty().onError(new TestException()); + fail("OnErrorNotImplementedException not thrown!"); + } catch (OnErrorNotImplementedException ex) { + if (!(ex.getCause() instanceof TestException)) { + fail("TestException not wrapped, instead: " + ex.getCause()); + } + } + } + @Test + public void testCreate1OnErrorNotImplemented() { + try { + Subscribers.create(Actions.empty()).onError(new TestException()); + fail("OnErrorNotImplementedException not thrown!"); + } catch (OnErrorNotImplementedException ex) { + if (!(ex.getCause() instanceof TestException)) { + fail("TestException not wrapped, instead: " + ex.getCause()); + } + } + } + @Test(expected = IllegalArgumentException.class) + public void testCreate1Null() { + Subscribers.create(null); + } + @Test(expected = IllegalArgumentException.class) + public void testCreate2Null() { + Action1 throwAction = Actions.empty(); + Subscribers.create(null, throwAction); + } + @Test(expected = IllegalArgumentException.class) + public void testCreate3Null() { + Subscribers.create(Actions.empty(), null); + } + + @Test(expected = IllegalArgumentException.class) + public void testCreate4Null() { + Action1 throwAction = Actions.empty(); + Subscribers.create(null, throwAction, Actions.empty()); + } + @Test(expected = IllegalArgumentException.class) + public void testCreate5Null() { + Subscribers.create(Actions.empty(), null, Actions.empty()); + } + @Test(expected = IllegalArgumentException.class) + public void testCreate6Null() { + Action1 throwAction = Actions.empty(); + Subscribers.create(Actions.empty(), throwAction, null); + } + + @Test + public void testCreate1Value() { + final AtomicInteger value = new AtomicInteger(); + Action1 action = new Action1() { + @Override + public void call(Integer t) { + value.set(t); + } + }; + Subscribers.create(action).onNext(1); + + assertEquals(1, value.get()); + } + @Test + public void testCreate2Value() { + final AtomicInteger value = new AtomicInteger(); + Action1 action = new Action1() { + @Override + public void call(Integer t) { + value.set(t); + } + }; + Action1 throwAction = Actions.empty(); + Subscribers.create(action, throwAction).onNext(1); + + assertEquals(1, value.get()); + } + + @Test + public void testCreate3Value() { + final AtomicInteger value = new AtomicInteger(); + Action1 action = new Action1() { + @Override + public void call(Integer t) { + value.set(t); + } + }; + Action1 throwAction = Actions.empty(); + Subscribers.create(action, throwAction, Actions.empty()).onNext(1); + + assertEquals(1, value.get()); + } + + @Test + public void testError2() { + final AtomicReference value = new AtomicReference(); + Action1 action = new Action1() { + @Override + public void call(Throwable t) { + value.set(t); + } + }; + TestException exception = new TestException(); + Subscribers.create(Actions.empty(), action).onError(exception); + + assertEquals(exception, value.get()); + } + + @Test + public void testError3() { + final AtomicReference value = new AtomicReference(); + Action1 action = new Action1() { + @Override + public void call(Throwable t) { + value.set(t); + } + }; + TestException exception = new TestException(); + Subscribers.create(Actions.empty(), action, Actions.empty()).onError(exception); + + assertEquals(exception, value.get()); + } + + @Test + public void testCompleted() { + Action0 action = mock(Action0.class); + + Action1 throwAction = Actions.empty(); + Subscribers.create(Actions.empty(), throwAction, action).onCompleted(); + + verify(action).call(); + } + @Test + public void testEmptyCompleted() { + Subscribers.create(Actions.empty()).onCompleted(); + + Action1 throwAction = Actions.empty(); + Subscribers.create(Actions.empty(), throwAction).onCompleted(); + } +} diff --git a/src/test/java/rx/observers/TestObserverTest.java b/src/test/java/rx/observers/TestObserverTest.java index aa253f2cd2..53f7a06746 100644 --- a/src/test/java/rx/observers/TestObserverTest.java +++ b/src/test/java/rx/observers/TestObserverTest.java @@ -15,20 +15,19 @@ */ package rx.observers; -import static org.junit.Assert.assertEquals; -import static org.mockito.Mockito.inOrder; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; -import java.util.Arrays; +import java.util.*; -import org.junit.Rule; -import org.junit.Test; +import org.junit.*; import org.junit.rules.ExpectedException; import org.mockito.InOrder; +import rx.Notification; import rx.Observable; import rx.Observer; +import rx.exceptions.TestException; import rx.subjects.PublishSubject; public class TestObserverTest { @@ -124,5 +123,109 @@ public void testWrappingMockWhenUnsubscribeInvolved() { public void testErrorSwallowed() { Observable.error(new RuntimeException()).subscribe(new TestObserver()); } + + @Test + public void testGetEvents() { + TestObserver to = new TestObserver(); + to.onNext(1); + to.onNext(2); + + assertEquals(Arrays.asList(Arrays.asList(1, 2), + Collections.emptyList(), + Collections.emptyList()), to.getEvents()); + + to.onCompleted(); + + assertEquals(Arrays.asList(Arrays.asList(1, 2), Collections.emptyList(), + Collections.singletonList(Notification.createOnCompleted())), to.getEvents()); + + TestException ex = new TestException(); + TestObserver to2 = new TestObserver(); + to2.onNext(1); + to2.onNext(2); + + assertEquals(Arrays.asList(Arrays.asList(1, 2), + Collections.emptyList(), + Collections.emptyList()), to2.getEvents()); + + to2.onError(ex); + + assertEquals(Arrays.asList( + Arrays.asList(1, 2), + Collections.singletonList(ex), + Collections.emptyList()), + to2.getEvents()); + } + @Test + public void testNullExpected() { + TestObserver to = new TestObserver(); + to.onNext(1); + + try { + to.assertReceivedOnNext(Arrays.asList((Integer)null)); + } catch (AssertionError ex) { + // this is expected + return; + } + fail("Null element check assertion didn't happen!"); + } + + @Test + public void testNullActual() { + TestObserver to = new TestObserver(); + to.onNext(null); + + try { + to.assertReceivedOnNext(Arrays.asList(1)); + } catch (AssertionError ex) { + // this is expected + return; + } + fail("Null element check assertion didn't happen!"); + } + + @Test + public void testTerminalErrorOnce() { + TestObserver to = new TestObserver(); + to.onError(new TestException()); + to.onError(new TestException()); + + try { + to.assertTerminalEvent(); + } catch (AssertionError ex) { + // this is expected + return; + } + fail("Failed to report multiple onError terminal events!"); + } + @Test + public void testTerminalCompletedOnce() { + TestObserver to = new TestObserver(); + to.onCompleted(); + to.onCompleted(); + + try { + to.assertTerminalEvent(); + } catch (AssertionError ex) { + // this is expected + return; + } + fail("Failed to report multiple onError terminal events!"); + } + + @Test + public void testTerminalOneKind() { + TestObserver to = new TestObserver(); + to.onError(new TestException()); + to.onCompleted(); + + try { + to.assertTerminalEvent(); + } catch (AssertionError ex) { + // this is expected + return; + } + fail("Failed to report multiple kinds of events!"); + } } diff --git a/src/test/java/rx/observers/TestSubscriberTest.java b/src/test/java/rx/observers/TestSubscriberTest.java index 75d59fc1f8..1076d2152f 100644 --- a/src/test/java/rx/observers/TestSubscriberTest.java +++ b/src/test/java/rx/observers/TestSubscriberTest.java @@ -15,25 +15,22 @@ */ package rx.observers; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.inOrder; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; import java.util.Arrays; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; +import org.junit.*; import org.junit.rules.ExpectedException; import org.mockito.InOrder; -import rx.Observable; -import rx.Observer; +import rx.*; +import rx.Scheduler.Worker; +import rx.exceptions.*; import rx.functions.Action0; +import rx.schedulers.Schedulers; import rx.subjects.PublishSubject; public class TestSubscriberTest { @@ -160,4 +157,442 @@ public void call() { assertTrue(unsub.get()); } + @Test(expected = NullPointerException.class) + public void testNullDelegate1() { + TestSubscriber ts = new TestSubscriber((Observer)null); + ts.onCompleted(); + } + + @Test(expected = NullPointerException.class) + public void testNullDelegate2() { + TestSubscriber ts = new TestSubscriber((Subscriber)null); + ts.onCompleted(); + } + + @Test(expected = NullPointerException.class) + public void testNullDelegate3() { + TestSubscriber ts = new TestSubscriber((Subscriber)null, 0); + ts.onCompleted(); + } + + @Test + public void testDelegate1() { + TestObserver to = new TestObserver(); + TestSubscriber ts = TestSubscriber.create(to); + ts.onCompleted(); + + to.assertTerminalEvent(); + } + + @Test + public void testDelegate2() { + TestSubscriber ts1 = TestSubscriber.create(); + TestSubscriber ts2 = TestSubscriber.create(ts1); + ts2.onCompleted(); + + ts1.assertCompleted(); + } + + @Test + public void testDelegate3() { + TestSubscriber ts1 = TestSubscriber.create(); + TestSubscriber ts2 = TestSubscriber.create(ts1, 0); + ts2.onCompleted(); + ts1.assertCompleted(); + } + + @Test + public void testUnsubscribed() { + TestSubscriber ts = new TestSubscriber(); + try { + ts.assertUnsubscribed(); + } catch (AssertionError ex) { + // expected + return; + } + fail("Not unsubscribed but not reported!"); + } + + @Test + public void testNoErrors() { + TestSubscriber ts = new TestSubscriber(); + ts.onError(new TestException()); + try { + ts.assertNoErrors(); + } catch (AssertionError ex) { + // expected + return; + } + fail("Error present but no assertion error!"); + } + + @Test + public void testNotCompleted() { + TestSubscriber ts = new TestSubscriber(); + try { + ts.assertCompleted(); + } catch (AssertionError ex) { + // expected + return; + } + fail("Not completed and no assertion error!"); + } + + @Test + public void testMultipleCompletions() { + TestSubscriber ts = new TestSubscriber(); + ts.onCompleted(); + ts.onCompleted(); + try { + ts.assertCompleted(); + } catch (AssertionError ex) { + // expected + return; + } + fail("Multiple completions and no assertion error!"); + } + + @Test + public void testCompleted() { + TestSubscriber ts = new TestSubscriber(); + ts.onCompleted(); + try { + ts.assertNotCompleted(); + } catch (AssertionError ex) { + // expected + return; + } + fail("Completed and no assertion error!"); + } + + @Test + public void testMultipleCompletions2() { + TestSubscriber ts = new TestSubscriber(); + ts.onCompleted(); + ts.onCompleted(); + try { + ts.assertNotCompleted(); + } catch (AssertionError ex) { + // expected + return; + } + fail("Multiple completions and no assertion error!"); + } + + @Test + public void testMultipleErrors() { + TestSubscriber ts = new TestSubscriber(); + ts.onError(new TestException()); + ts.onError(new TestException()); + try { + ts.assertNoErrors(); + } catch (AssertionError ex) { + if (!(ex.getCause() instanceof CompositeException)) { + fail("Multiple Error present but the reported error doesn't have a composite cause!"); + } + // expected + return; + } + fail("Multiple Error present but no assertion error!"); + } + + @Test + public void testMultipleErrors2() { + TestSubscriber ts = new TestSubscriber(); + ts.onError(new TestException()); + ts.onError(new TestException()); + try { + ts.assertError(TestException.class); + } catch (AssertionError ex) { + if (!(ex.getCause() instanceof CompositeException)) { + fail("Multiple Error present but the reported error doesn't have a composite cause!"); + } + // expected + return; + } + fail("Multiple Error present but no assertion error!"); + } + + @Test + public void testMultipleErrors3() { + TestSubscriber ts = new TestSubscriber(); + ts.onError(new TestException()); + ts.onError(new TestException()); + try { + ts.assertError(new TestException()); + } catch (AssertionError ex) { + if (!(ex.getCause() instanceof CompositeException)) { + fail("Multiple Error present but the reported error doesn't have a composite cause!"); + } + // expected + return; + } + fail("Multiple Error present but no assertion error!"); + } + + @Test + public void testDifferentError() { + TestSubscriber ts = new TestSubscriber(); + ts.onError(new TestException()); + try { + ts.assertError(new TestException()); + } catch (AssertionError ex) { + // expected + return; + } + fail("Different Error present but no assertion error!"); + } + + @Test + public void testDifferentError2() { + TestSubscriber ts = new TestSubscriber(); + ts.onError(new RuntimeException()); + try { + ts.assertError(new TestException()); + } catch (AssertionError ex) { + // expected + return; + } + fail("Different Error present but no assertion error!"); + } + + @Test + public void testDifferentError3() { + TestSubscriber ts = new TestSubscriber(); + ts.onError(new RuntimeException()); + try { + ts.assertError(TestException.class); + } catch (AssertionError ex) { + // expected + return; + } + fail("Different Error present but no assertion error!"); + } + + @Test + public void testNoError() { + TestSubscriber ts = new TestSubscriber(); + try { + ts.assertError(TestException.class); + } catch (AssertionError ex) { + // expected + return; + } + fail("No present but no assertion error!"); + } + + @Test + public void testNoError2() { + TestSubscriber ts = new TestSubscriber(); + try { + ts.assertError(new TestException()); + } catch (AssertionError ex) { + // expected + return; + } + fail("No present but no assertion error!"); + } + + @Test + public void testInterruptTerminalEventAwait() { + TestSubscriber ts = TestSubscriber.create(); + + final Thread t0 = Thread.currentThread(); + Worker w = Schedulers.computation().createWorker(); + try { + w.schedule(new Action0() { + @Override + public void call() { + t0.interrupt(); + } + }, 200, TimeUnit.MILLISECONDS); + + try { + ts.awaitTerminalEvent(); + fail("Did not interrupt wait!"); + } catch (RuntimeException ex) { + if (!(ex.getCause() instanceof InterruptedException)) { + fail("The cause is not InterruptedException! " + ex.getCause()); + } + } + } finally { + w.unsubscribe(); + } + } + + @Test + public void testInterruptTerminalEventAwaitTimed() { + TestSubscriber ts = TestSubscriber.create(); + + final Thread t0 = Thread.currentThread(); + Worker w = Schedulers.computation().createWorker(); + try { + w.schedule(new Action0() { + @Override + public void call() { + t0.interrupt(); + } + }, 200, TimeUnit.MILLISECONDS); + + try { + ts.awaitTerminalEvent(5, TimeUnit.SECONDS); + fail("Did not interrupt wait!"); + } catch (RuntimeException ex) { + if (!(ex.getCause() instanceof InterruptedException)) { + fail("The cause is not InterruptedException! " + ex.getCause()); + } + } + } finally { + w.unsubscribe(); + } + } + + @Test + public void testInterruptTerminalEventAwaitAndUnsubscribe() { + TestSubscriber ts = TestSubscriber.create(); + + final Thread t0 = Thread.currentThread(); + Worker w = Schedulers.computation().createWorker(); + try { + w.schedule(new Action0() { + @Override + public void call() { + t0.interrupt(); + } + }, 200, TimeUnit.MILLISECONDS); + + ts.awaitTerminalEventAndUnsubscribeOnTimeout(5, TimeUnit.SECONDS); + if (!ts.isUnsubscribed()) { + fail("Did not unsubscribe!"); + } + } finally { + w.unsubscribe(); + } + } + + @Test + public void testNoTerminalEventBut1Completed() { + TestSubscriber ts = TestSubscriber.create(); + + ts.onCompleted(); + + try { + ts.assertNoTerminalEvent(); + fail("Failed to report there were terminal event(s)!"); + } catch (AssertionError ex) { + // expected + } + } + + @Test + public void testNoTerminalEventBut1Error() { + TestSubscriber ts = TestSubscriber.create(); + + ts.onError(new TestException()); + + try { + ts.assertNoTerminalEvent(); + fail("Failed to report there were terminal event(s)!"); + } catch (AssertionError ex) { + // expected + } + } + + @Test + public void testNoTerminalEventBut1Error1Completed() { + TestSubscriber ts = TestSubscriber.create(); + + ts.onCompleted(); + ts.onError(new TestException()); + + try { + ts.assertNoTerminalEvent(); + fail("Failed to report there were terminal event(s)!"); + } catch (AssertionError ex) { + // expected + } + } + + @Test + public void testNoTerminalEventBut2Errors() { + TestSubscriber ts = TestSubscriber.create(); + + ts.onError(new TestException()); + ts.onError(new TestException()); + + try { + ts.assertNoTerminalEvent(); + fail("Failed to report there were terminal event(s)!"); + } catch (AssertionError ex) { + // expected + if (!(ex.getCause() instanceof CompositeException)) { + fail("Did not report a composite exception cause: " + ex.getCause()); + } + } + } + + @Test + public void testNoValues() { + TestSubscriber ts = TestSubscriber.create(); + ts.onNext(1); + + try { + ts.assertNoValues(); + fail("Failed to report there were values!"); + } catch (AssertionError ex) { + // expected + } + } + + @Test + public void testValueCount() { + TestSubscriber ts = TestSubscriber.create(); + ts.onNext(1); + ts.onNext(2); + + try { + ts.assertValueCount(3); + fail("Failed to report there were values!"); + } catch (AssertionError ex) { + // expected + } + } + + @Test(timeout = 1000) + public void testOnCompletedCrashCountsDownLatch() { + TestObserver to = new TestObserver() { + @Override + public void onCompleted() { + throw new TestException(); + } + }; + TestSubscriber ts = TestSubscriber.create(to); + + try { + ts.onCompleted(); + } catch (TestException ex) { + // expected + } + + ts.awaitTerminalEvent(); + } + + @Test(timeout = 1000) + public void testOnErrorCrashCountsDownLatch() { + TestObserver to = new TestObserver() { + @Override + public void onError(Throwable e) { + throw new TestException(); + } + }; + TestSubscriber ts = TestSubscriber.create(to); + + try { + ts.onError(new RuntimeException()); + } catch (TestException ex) { + // expected + } + + ts.awaitTerminalEvent(); + } } From 25bbcf1ebe1e4df3e6a52fa72f876c4beb43271a Mon Sep 17 00:00:00 2001 From: Dave Moten Date: Mon, 13 Jul 2015 16:02:22 +1000 Subject: [PATCH 36/81] OperatorSwitch - fix lost requests race condition using ProducerArbiter --- .../rx/internal/operators/OperatorSwitch.java | 150 +++++++----------- .../operators/OperatorSwitchIfEmptyTest.java | 1 - .../operators/OperatorSwitchTest.java | 19 +-- 3 files changed, 63 insertions(+), 107 deletions(-) diff --git a/src/main/java/rx/internal/operators/OperatorSwitch.java b/src/main/java/rx/internal/operators/OperatorSwitch.java index afd35e477d..cbd02e1b58 100644 --- a/src/main/java/rx/internal/operators/OperatorSwitch.java +++ b/src/main/java/rx/internal/operators/OperatorSwitch.java @@ -22,6 +22,7 @@ import rx.Observable.Operator; import rx.Producer; import rx.Subscriber; +import rx.internal.producers.ProducerArbiter; import rx.observers.SerializedSubscriber; import rx.subscriptions.SerialSubscription; @@ -46,7 +47,9 @@ private static final class Holder { public static OperatorSwitch instance() { return (OperatorSwitch)Holder.INSTANCE; } + private OperatorSwitch() { } + @Override public Subscriber> call(final Subscriber child) { SwitchSubscriber sws = new SwitchSubscriber(child); @@ -55,10 +58,12 @@ public Subscriber> call(final Subscriber extends Subscriber> { - final SerializedSubscriber s; + final SerializedSubscriber serializedChild; final SerialSubscription ssub; final Object guard = new Object(); final NotificationLite nl = NotificationLite.instance(); + final ProducerArbiter arbiter; + /** Guarded by guard. */ int index; /** Guarded by guard. */ @@ -70,50 +75,19 @@ private static final class SwitchSubscriber extends Subscriber currentSubscriber; - public SwitchSubscriber(Subscriber child) { - s = new SerializedSubscriber(child); + SwitchSubscriber(Subscriber child) { + serializedChild = new SerializedSubscriber(child); + arbiter = new ProducerArbiter(); ssub = new SerialSubscription(); child.add(ssub); child.setProducer(new Producer(){ @Override public void request(long n) { - if (infinite) { - return; - } - if(n == Long.MAX_VALUE) { - infinite = true; - } - InnerSubscriber localSubscriber; - synchronized (guard) { - localSubscriber = currentSubscriber; - if (currentSubscriber == null) { - long r = initialRequested + n; - if (r < 0) { - infinite = true; - } else { - initialRequested = r; - } - } else { - long r = currentSubscriber.requested + n; - if (r < 0) { - infinite = true; - } else { - currentSubscriber.requested = r; - } - } - } - if (localSubscriber != null) { - if (infinite) - localSubscriber.requestMore(Long.MAX_VALUE); - else - localSubscriber.requestMore(n); + if (n > 0) { + arbiter.request(n); } } }); @@ -122,26 +96,18 @@ public void request(long n) { @Override public void onNext(Observable t) { final int id; - long remainingRequest; synchronized (guard) { id = ++index; active = true; - if (infinite) { - remainingRequest = Long.MAX_VALUE; - } else { - remainingRequest = currentSubscriber == null ? initialRequested : currentSubscriber.requested; - } - currentSubscriber = new InnerSubscriber(id, remainingRequest); - currentSubscriber.requested = remainingRequest; + currentSubscriber = new InnerSubscriber(id, arbiter, this); } ssub.set(currentSubscriber); - t.unsafeSubscribe(currentSubscriber); } @Override public void onError(Throwable e) { - s.onError(e); + serializedChild.onError(e); unsubscribe(); } @@ -165,10 +131,10 @@ public void onCompleted() { emitting = true; } drain(localQueue); - s.onCompleted(); + serializedChild.onCompleted(); unsubscribe(); } - void emit(T value, int id, InnerSubscriber innerSubscriber) { + void emit(T value, int id, InnerSubscriber innerSubscriber) { List localQueue; synchronized (guard) { if (id != index) { @@ -178,8 +144,6 @@ void emit(T value, int id, InnerSubscriber innerSubscriber) { if (queue == null) { queue = new ArrayList(); } - if (innerSubscriber.requested != Long.MAX_VALUE) - innerSubscriber.requested--; queue.add(value); return; } @@ -194,11 +158,8 @@ void emit(T value, int id, InnerSubscriber innerSubscriber) { drain(localQueue); if (once) { once = false; - synchronized (guard) { - if (innerSubscriber.requested != Long.MAX_VALUE) - innerSubscriber.requested--; - } - s.onNext(value); + serializedChild.onNext(value); + arbiter.produced(1); } synchronized (guard) { localQueue = queue; @@ -209,7 +170,7 @@ void emit(T value, int id, InnerSubscriber innerSubscriber) { break; } } - } while (!s.isUnsubscribed()); + } while (!serializedChild.isUnsubscribed()); } finally { if (!skipFinal) { synchronized (guard) { @@ -224,16 +185,17 @@ void drain(List localQueue) { } for (Object o : localQueue) { if (nl.isCompleted(o)) { - s.onCompleted(); + serializedChild.onCompleted(); break; } else if (nl.isError(o)) { - s.onError(nl.getError(o)); + serializedChild.onError(nl.getError(o)); break; } else { @SuppressWarnings("unchecked") T t = (T)o; - s.onNext(t); + serializedChild.onNext(t); + arbiter.produced(1); } } } @@ -258,7 +220,7 @@ void error(Throwable e, int id) { } drain(localQueue); - s.onError(e); + serializedChild.onError(e); unsubscribe(); } void complete(int id) { @@ -285,51 +247,45 @@ void complete(int id) { } drain(localQueue); - s.onCompleted(); + serializedChild.onCompleted(); unsubscribe(); } - final class InnerSubscriber extends Subscriber { - - /** - * The number of request that is not acknowledged. - * - * Guarded by guard. - */ - private long requested = 0; - - private final int id; + } + + private static final class InnerSubscriber extends Subscriber { - private final long initialRequested; + private final int id; - public InnerSubscriber(int id, long initialRequested) { - this.id = id; - this.initialRequested = initialRequested; - } + private final ProducerArbiter arbiter; - @Override - public void onStart() { - requestMore(initialRequested); - } + private final SwitchSubscriber parent; - public void requestMore(long n) { - request(n); - } + InnerSubscriber(int id, ProducerArbiter arbiter, SwitchSubscriber parent) { + this.id = id; + this.arbiter = arbiter; + this.parent = parent; + } + + @Override + public void setProducer(Producer p) { + arbiter.setProducer(p); + } - @Override - public void onNext(T t) { - emit(t, id, this); - } + @Override + public void onNext(T t) { + parent.emit(t, id, this); + } - @Override - public void onError(Throwable e) { - error(e, id); - } + @Override + public void onError(Throwable e) { + parent.error(e, id); + } - @Override - public void onCompleted() { - complete(id); - } + @Override + public void onCompleted() { + parent.complete(id); } } + } diff --git a/src/test/java/rx/internal/operators/OperatorSwitchIfEmptyTest.java b/src/test/java/rx/internal/operators/OperatorSwitchIfEmptyTest.java index 2534613ab4..332924ba68 100644 --- a/src/test/java/rx/internal/operators/OperatorSwitchIfEmptyTest.java +++ b/src/test/java/rx/internal/operators/OperatorSwitchIfEmptyTest.java @@ -27,7 +27,6 @@ import rx.Observable; import rx.Observable.OnSubscribe; import rx.functions.Action0; -import rx.functions.Action1; import rx.observers.TestSubscriber; import rx.schedulers.Schedulers; import rx.subscriptions.Subscriptions; diff --git a/src/test/java/rx/internal/operators/OperatorSwitchTest.java b/src/test/java/rx/internal/operators/OperatorSwitchTest.java index 6b5d3a1f79..63de5d0d81 100644 --- a/src/test/java/rx/internal/operators/OperatorSwitchTest.java +++ b/src/test/java/rx/internal/operators/OperatorSwitchTest.java @@ -25,7 +25,6 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.concurrent.CopyOnWriteArrayList; @@ -642,32 +641,34 @@ public Observable call(Long t) { } @Test(timeout = 10000) - public void testSecondaryRequestsAdditivelyAreMoreThanLongMaxValueInducesMaxValueRequestFromUpstream() throws InterruptedException { + public void testSecondaryRequestsAdditivelyAreMoreThanLongMaxValueInducesMaxValueRequestFromUpstream() + throws InterruptedException { final List requests = new CopyOnWriteArrayList(); final Action1 addRequest = new Action1() { @Override public void call(Long n) { requests.add(n); - }}; - TestSubscriber ts = new TestSubscriber(0); + } + }; + TestSubscriber ts = new TestSubscriber(1); Observable.switchOnNext( Observable.interval(100, TimeUnit.MILLISECONDS) .map(new Func1>() { @Override public Observable call(Long t) { - return Observable.from(Arrays.asList(1L, 2L, 3L)).doOnRequest(addRequest); + return Observable.from(Arrays.asList(1L, 2L, 3L)).doOnRequest( + addRequest); } }).take(3)).subscribe(ts); - ts.requestMore(1); - //we will miss two of the first observable + // we will miss two of the first observables Thread.sleep(250); ts.requestMore(Long.MAX_VALUE - 1); ts.requestMore(Long.MAX_VALUE - 1); ts.awaitTerminalEvent(); assertTrue(ts.getOnNextEvents().size() > 0); assertEquals(5, (int) requests.size()); - assertEquals(Long.MAX_VALUE, (long) requests.get(3)); - assertEquals(Long.MAX_VALUE, (long) requests.get(4)); + assertEquals(Long.MAX_VALUE, (long) requests.get(requests.size()-1)); } + } From adbbd61bd6b24dfeb4e4d7f49f8790ea0ecc384b Mon Sep 17 00:00:00 2001 From: akarnokd Date: Sat, 8 Aug 2015 01:47:25 +0200 Subject: [PATCH 37/81] FromIterable overhead reduction. --- .../operators/OnSubscribeFromIterable.java | 105 ++++++++++-------- .../java/rx/operators/FromIterablePerf.java | 85 ++++++++++++++ 2 files changed, 142 insertions(+), 48 deletions(-) create mode 100644 src/perf/java/rx/operators/FromIterablePerf.java diff --git a/src/main/java/rx/internal/operators/OnSubscribeFromIterable.java b/src/main/java/rx/internal/operators/OnSubscribeFromIterable.java index 2aad771b57..f4790e75bd 100644 --- a/src/main/java/rx/internal/operators/OnSubscribeFromIterable.java +++ b/src/main/java/rx/internal/operators/OnSubscribeFromIterable.java @@ -16,11 +16,10 @@ package rx.internal.operators; import java.util.Iterator; -import java.util.concurrent.atomic.AtomicLongFieldUpdater; +import java.util.concurrent.atomic.AtomicLong; +import rx.*; import rx.Observable.OnSubscribe; -import rx.Producer; -import rx.Subscriber; /** * Converts an {@code Iterable} sequence into an {@code Observable}. @@ -50,14 +49,12 @@ public void call(final Subscriber o) { o.setProducer(new IterableProducer(o, it)); } - private static final class IterableProducer implements Producer { + private static final class IterableProducer extends AtomicLong implements Producer { + /** */ + private static final long serialVersionUID = -8730475647105475802L; private final Subscriber o; private final Iterator it; - private volatile long requested = 0; - @SuppressWarnings("rawtypes") - private static final AtomicLongFieldUpdater REQUESTED_UPDATER = AtomicLongFieldUpdater.newUpdater(IterableProducer.class, "requested"); - private IterableProducer(Subscriber o, Iterator it) { this.o = o; this.it = it; @@ -65,18 +62,41 @@ private IterableProducer(Subscriber o, Iterator it) { @Override public void request(long n) { - if (requested == Long.MAX_VALUE) { + if (get() == Long.MAX_VALUE) { // already started with fast-path return; } - if (n == Long.MAX_VALUE && REQUESTED_UPDATER.compareAndSet(this, 0, Long.MAX_VALUE)) { - // fast-path without backpressure + if (n == Long.MAX_VALUE && compareAndSet(0, Long.MAX_VALUE)) { + fastpath(); + } else + if (n > 0 && BackpressureUtils.getAndAddRequest(this, n) == 0L) { + slowpath(n); + } + + } + + void slowpath(long n) { + // backpressure is requested + final Subscriber o = this.o; + final Iterator it = this.it; + long r = n; + while (true) { + /* + * This complicated logic is done to avoid touching the + * volatile `requested` value during the loop itself. If + * it is touched during the loop the performance is + * impacted significantly. + */ + long numToEmit = r; while (true) { if (o.isUnsubscribed()) { return; } else if (it.hasNext()) { - o.onNext(it.next()); + if (--numToEmit >= 0) { + o.onNext(it.next()); + } else + break; } else if (!o.isUnsubscribed()) { o.onCompleted(); return; @@ -85,45 +105,34 @@ public void request(long n) { return; } } - } else if (n > 0) { - // backpressure is requested - long _c = BackpressureUtils.getAndAddRequest(REQUESTED_UPDATER, this, n); - if (_c == 0) { - while (true) { - /* - * This complicated logic is done to avoid touching the - * volatile `requested` value during the loop itself. If - * it is touched during the loop the performance is - * impacted significantly. - */ - long r = requested; - long numToEmit = r; - while (true) { - if (o.isUnsubscribed()) { - return; - } else if (it.hasNext()) { - if (--numToEmit >= 0) { - o.onNext(it.next()); - } else - break; - } else if (!o.isUnsubscribed()) { - o.onCompleted(); - return; - } else { - // is unsubscribed - return; - } - } - if (REQUESTED_UPDATER.addAndGet(this, -r) == 0) { - // we're done emitting the number requested so - // return - return; - } - - } + r = addAndGet(-r); + if (r == 0L) { + // we're done emitting the number requested so + // return + return; } + } + } + void fastpath() { + // fast-path without backpressure + final Subscriber o = this.o; + final Iterator it = this.it; + + while (true) { + if (o.isUnsubscribed()) { + return; + } else if (it.hasNext()) { + o.onNext(it.next()); + } else if (!o.isUnsubscribed()) { + o.onCompleted(); + return; + } else { + // is unsubscribed + return; + } + } } } diff --git a/src/perf/java/rx/operators/FromIterablePerf.java b/src/perf/java/rx/operators/FromIterablePerf.java new file mode 100644 index 0000000000..1368fcbf30 --- /dev/null +++ b/src/perf/java/rx/operators/FromIterablePerf.java @@ -0,0 +1,85 @@ +/** + * Copyright 2014 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package rx.operators; + +import java.util.Arrays; +import java.util.concurrent.TimeUnit; + +import org.openjdk.jmh.annotations.*; +import org.openjdk.jmh.infra.Blackhole; + +import rx.*; +import rx.internal.operators.OnSubscribeFromIterable; +import rx.jmh.LatchedObserver; + +/** + * Benchmark from(Iterable). + *

+ * gradlew benchmarks "-Pjmh=-f 1 -tu s -bm thrpt -wi 5 -i 5 -r 1 .*FromIterablePerf.*" + *

+ * gradlew benchmarks "-Pjmh=-f 1 -tu ns -bm avgt -wi 5 -i 5 -r 1 .*FromIterablePerf.*" + */ +@BenchmarkMode(Mode.Throughput) +@OutputTimeUnit(TimeUnit.SECONDS) +@State(Scope.Thread) +public class FromIterablePerf { + Observable from; + OnSubscribeFromIterable direct; + @Param({"1", "1000", "1000000"}) + public int size; + + @Setup + public void setup() { + Integer[] array = new Integer[size]; + for (int i = 0; i < size; i++) { + array[i] = i; + } + from = Observable.from(Arrays.asList(array)); + direct = new OnSubscribeFromIterable(Arrays.asList(array)); + } + + @Benchmark + public void from(Blackhole bh) { + from.subscribe(new LatchedObserver(bh)); + } + @Benchmark + public void fromUnsafe(final Blackhole bh) { + from.unsafeSubscribe(createSubscriber(bh)); + } + + @Benchmark + public void direct(final Blackhole bh) { + direct.call(createSubscriber(bh)); + } + + Subscriber createSubscriber(final Blackhole bh) { + return new Subscriber() { + @Override + public void onNext(Integer t) { + bh.consume(t); + } + @Override + public void onError(Throwable e) { + e.printStackTrace(); + } + @Override + public void onCompleted() { + + } + }; + } +} From 55dccd023f196cb706f5c74452a3cf677df2311b Mon Sep 17 00:00:00 2001 From: akarnokd Date: Tue, 11 Aug 2015 16:39:29 +0200 Subject: [PATCH 38/81] Range overhead reduction --- .../internal/operators/OnSubscribeRange.java | 112 +++++++++++------- .../java/rx/operators/OperatorRangePerf.java | 17 +-- .../operators/OnSubscribeRangeTest.java | 19 +++ 3 files changed, 92 insertions(+), 56 deletions(-) diff --git a/src/main/java/rx/internal/operators/OnSubscribeRange.java b/src/main/java/rx/internal/operators/OnSubscribeRange.java index bcfbe0736b..383d17f28f 100644 --- a/src/main/java/rx/internal/operators/OnSubscribeRange.java +++ b/src/main/java/rx/internal/operators/OnSubscribeRange.java @@ -15,11 +15,10 @@ */ package rx.internal.operators; -import java.util.concurrent.atomic.AtomicLongFieldUpdater; +import java.util.concurrent.atomic.AtomicLong; +import rx.*; import rx.Observable.OnSubscribe; -import rx.Producer; -import rx.Subscriber; /** * Emit ints from start to end inclusive. @@ -39,13 +38,13 @@ public void call(final Subscriber o) { o.setProducer(new RangeProducer(o, start, end)); } - private static final class RangeProducer implements Producer { + private static final class RangeProducer extends AtomicLong implements Producer { + /** */ + private static final long serialVersionUID = 4114392207069098388L; + private final Subscriber o; - // accessed by REQUESTED_UPDATER - private volatile long requested; - private static final AtomicLongFieldUpdater REQUESTED_UPDATER = AtomicLongFieldUpdater.newUpdater(RangeProducer.class, "requested"); - private long index; private final int end; + private long index; private RangeProducer(Subscriber o, int start, int end) { this.o = o; @@ -55,54 +54,79 @@ private RangeProducer(Subscriber o, int start, int end) { @Override public void request(long n) { - if (requested == Long.MAX_VALUE) { + if (get() == Long.MAX_VALUE) { // already started with fast-path return; } - if (n == Long.MAX_VALUE && REQUESTED_UPDATER.compareAndSet(this, 0, Long.MAX_VALUE)) { + if (n == Long.MAX_VALUE && compareAndSet(0L, Long.MAX_VALUE)) { // fast-path without backpressure - for (long i = index; i <= end; i++) { + fastpath(); + } else if (n > 0L) { + long c = BackpressureUtils.getAndAddRequest(this, n); + if (c == 0L) { + // backpressure is requested + slowpath(n); + } + } + } + + /** + * + */ + void slowpath(long r) { + long idx = index; + while (true) { + /* + * This complicated logic is done to avoid touching the volatile `index` and `requested` values + * during the loop itself. If they are touched during the loop the performance is impacted significantly. + */ + long fs = end - idx + 1; + long e = Math.min(fs, r); + final boolean complete = fs <= r; + + fs = e + idx; + final Subscriber o = this.o; + + for (long i = idx; i != fs; i++) { if (o.isUnsubscribed()) { return; } o.onNext((int) i); } - if (!o.isUnsubscribed()) { + + if (complete) { + if (o.isUnsubscribed()) { + return; + } o.onCompleted(); + return; } - } else if (n > 0) { - // backpressure is requested - long _c = BackpressureUtils.getAndAddRequest(REQUESTED_UPDATER,this, n); - if (_c == 0) { - while (true) { - /* - * This complicated logic is done to avoid touching the volatile `index` and `requested` values - * during the loop itself. If they are touched during the loop the performance is impacted significantly. - */ - long r = requested; - long idx = index; - long numLeft = end - idx + 1; - long e = Math.min(numLeft, r); - boolean completeOnFinish = numLeft <= r; - long stopAt = e + idx; - for (long i = idx; i < stopAt; i++) { - if (o.isUnsubscribed()) { - return; - } - o.onNext((int) i); - } - index = stopAt; - - if (completeOnFinish) { - o.onCompleted(); - return; - } - if (REQUESTED_UPDATER.addAndGet(this, -e) == 0) { - // we're done emitting the number requested so return - return; - } - } + + idx = fs; + index = fs; + + r = addAndGet(-e); + if (r == 0L) { + // we're done emitting the number requested so return + return; + } + } + } + + /** + * + */ + void fastpath() { + final long end = this.end + 1L; + final Subscriber o = this.o; + for (long i = index; i != end; i++) { + if (o.isUnsubscribed()) { + return; } + o.onNext((int) i); + } + if (!o.isUnsubscribed()) { + o.onCompleted(); } } } diff --git a/src/perf/java/rx/operators/OperatorRangePerf.java b/src/perf/java/rx/operators/OperatorRangePerf.java index 85ca76e46d..52fd0af7ff 100644 --- a/src/perf/java/rx/operators/OperatorRangePerf.java +++ b/src/perf/java/rx/operators/OperatorRangePerf.java @@ -17,18 +17,11 @@ import java.util.concurrent.TimeUnit; -import org.openjdk.jmh.annotations.Benchmark; -import org.openjdk.jmh.annotations.BenchmarkMode; -import org.openjdk.jmh.annotations.Mode; -import org.openjdk.jmh.annotations.OutputTimeUnit; -import org.openjdk.jmh.annotations.Param; -import org.openjdk.jmh.annotations.Scope; -import org.openjdk.jmh.annotations.Setup; -import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.*; import org.openjdk.jmh.infra.Blackhole; -import rx.Observable; -import rx.Subscriber; +import rx.*; +import rx.internal.operators.OnSubscribeRange; @BenchmarkMode(Mode.Throughput) @OutputTimeUnit(TimeUnit.SECONDS) @@ -50,7 +43,7 @@ public static class InputUsingRequest { @Setup public void setup(final Blackhole bh) { - observable = Observable.range(0, size); + observable = Observable.create(new OnSubscribeRange(0, size)); this.bh = bh; } @@ -98,7 +91,7 @@ public static class InputWithoutRequest { @Setup public void setup(final Blackhole bh) { - observable = Observable.range(0, size); + observable = Observable.create(new OnSubscribeRange(0, size)); this.bh = bh; } diff --git a/src/test/java/rx/internal/operators/OnSubscribeRangeTest.java b/src/test/java/rx/internal/operators/OnSubscribeRangeTest.java index 6d4d97d019..acc2f6ff75 100644 --- a/src/test/java/rx/internal/operators/OnSubscribeRangeTest.java +++ b/src/test/java/rx/internal/operators/OnSubscribeRangeTest.java @@ -249,4 +249,23 @@ public void onNext(Integer t) { }}); assertTrue(completed.get()); } + + @Test(timeout = 1000) + public void testNearMaxValueWithoutBackpressure() { + TestSubscriber ts = TestSubscriber.create(); + Observable.range(Integer.MAX_VALUE - 1, 2).subscribe(ts); + + ts.assertCompleted(); + ts.assertNoErrors(); + ts.assertValues(Integer.MAX_VALUE - 1, Integer.MAX_VALUE); + } + @Test(timeout = 1000) + public void testNearMaxValueWithBackpressure() { + TestSubscriber ts = TestSubscriber.create(3); + Observable.range(Integer.MAX_VALUE - 1, 2).subscribe(ts); + + ts.assertCompleted(); + ts.assertNoErrors(); + ts.assertValues(Integer.MAX_VALUE - 1, Integer.MAX_VALUE); + } } From 996f366395e9b9e69d30ca2c7c85dec6d81d7d36 Mon Sep 17 00:00:00 2001 From: Artem Zinnatullin Date: Mon, 10 Aug 2015 19:08:13 +0300 Subject: [PATCH 39/81] Remove redundant final modifier from static method in Actions --- src/main/java/rx/functions/Actions.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/rx/functions/Actions.java b/src/main/java/rx/functions/Actions.java index 342cfd030c..862bba221c 100644 --- a/src/main/java/rx/functions/Actions.java +++ b/src/main/java/rx/functions/Actions.java @@ -24,7 +24,7 @@ private Actions() { } @SuppressWarnings("unchecked") - public static final EmptyAction empty() { + public static EmptyAction empty() { return EMPTY_ACTION; } From 6808ce9e68b99248de5f03447bcd81c0261d2671 Mon Sep 17 00:00:00 2001 From: Steve Gury Date: Wed, 12 Aug 2015 15:32:25 -0700 Subject: [PATCH 40/81] Version 1.0.14 --- CHANGES.md | 29 +++++++++++++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/CHANGES.md b/CHANGES.md index 3f6776cd46..5a443e1a37 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,5 +1,34 @@ # RxJava Releases # +### Version 1.0.14 – August 12th 2015 ([Maven Central](http://search.maven.org/#artifactdetails%7Cio.reactivex%7Crxjava%7C1.0.14%7C)) ### + +* [Pull 2963] (https://github.com/ReactiveX/RxJava/pull/2963) Set of standard producers and updated queue implementations +* [Pull 3138] (https://github.com/ReactiveX/RxJava/pull/3138) Range overhead reduction. +* [Pull 3137] (https://github.com/ReactiveX/RxJava/pull/3137) FromIterable overhead reduction. +* [Pull 3078] (https://github.com/ReactiveX/RxJava/pull/3078) switchOnNext() - fix lost requests race condition +* [Pull 3112] (https://github.com/ReactiveX/RxJava/pull/3112) Observers package test coverage and fixes. +* [Pull 3123] (https://github.com/ReactiveX/RxJava/pull/3123) Remove redundant type parameter in EmptyAction +* [Pull 3104] (https://github.com/ReactiveX/RxJava/pull/3104) Fix SynchronizedQueue.equals +* [Pull 3147] (https://github.com/ReactiveX/RxJava/pull/3147) Remove unnecessary static modifier +* [Pull 3144] (https://github.com/ReactiveX/RxJava/pull/3144) Remove redundant cast in Exceptions +* [Pull 3143] (https://github.com/ReactiveX/RxJava/pull/3143) Fix for BackpressureUtils method javadoc +* [Pull 3141] (https://github.com/ReactiveX/RxJava/pull/3141) Improved Scheduler.Worker memory leak detection +* [Pull 3082] (https://github.com/ReactiveX/RxJava/pull/3082) Observable.x(ConversionFunc) to allow extensions to Observables +* [Pull 3103] (https://github.com/ReactiveX/RxJava/pull/3103) materialize() - add backpressure support +* [Pull 3129] (https://github.com/ReactiveX/RxJava/pull/3129) Fix retry with predicate ignoring backpressure. +* [Pull 3121] (https://github.com/ReactiveX/RxJava/pull/3121) Improve performance of NewThreadWorker, disable search for setRemoveOnCancelPolicy() on Android API < 21 +* [Pull 3120] (https://github.com/ReactiveX/RxJava/pull/3120) No InterruptedException with synchronous BlockingObservable +* [Pull 3117] (https://github.com/ReactiveX/RxJava/pull/3117) Operator replay() now supports backpressure +* [Pull 3116] (https://github.com/ReactiveX/RxJava/pull/3116) cache() now supports backpressure +* [Pull 3110] (https://github.com/ReactiveX/RxJava/pull/3110) Test coverage of rx.functions utility methods. +* [Pull 3101] (https://github.com/ReactiveX/RxJava/pull/3101) Fix take swallowing exception if thrown by exactly the nth onNext call to it. +* [Pull 3109] (https://github.com/ReactiveX/RxJava/pull/3109) Unit tests and cleanup of JCTools' queues. +* [Pull 3108] (https://github.com/ReactiveX/RxJava/pull/3108) remove OperatorOnErrorFlatMap because unused +* [Pull 3079] (https://github.com/ReactiveX/RxJava/pull/3079) fix forEach javadoc +* [Pull 3085] (https://github.com/ReactiveX/RxJava/pull/3085) break tests as approach timeout so that don't fail on slow machines +* [Pull 3086] (https://github.com/ReactiveX/RxJava/pull/3086) improve ExecutorSchedulerTest.testOnBackpressureDrop +* [Pull 3093] (https://github.com/ReactiveX/RxJava/pull/3093) Fix request != 0 checking in the scalar paths of merge() + ### Version 1.0.13 – July 20th 2015 ([Maven Central](http://search.maven.org/#artifactdetails%7Cio.reactivex%7Crxjava%7C1.0.13%7C)) ### This release has quite a few bug fixes and some new functionality. Items of note are detailed here with the list of changes at the bottom. From 3b8976d9d5c7fed473c7c21f97f0686acf4d315e Mon Sep 17 00:00:00 2001 From: Ben Christensen Date: Thu, 20 Aug 2015 20:56:25 -0700 Subject: [PATCH 41/81] Update README.md Add StackOverflow link --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index 9014a45c7e..5dcb2199a8 100644 --- a/README.md +++ b/README.md @@ -24,6 +24,7 @@ Learn more about RxJava on the " + pluginException.getMessage()); + pluginException.printStackTrace(); + } + +} diff --git a/src/main/java/rx/observers/SafeSubscriber.java b/src/main/java/rx/observers/SafeSubscriber.java index 0181887c34..8a9aad5179 100644 --- a/src/main/java/rx/observers/SafeSubscriber.java +++ b/src/main/java/rx/observers/SafeSubscriber.java @@ -20,9 +20,11 @@ import rx.Subscriber; import rx.exceptions.CompositeException; import rx.exceptions.Exceptions; +import rx.exceptions.OnCompletedFailedException; import rx.exceptions.OnErrorFailedException; import rx.exceptions.OnErrorNotImplementedException; -import rx.plugins.RxJavaPlugins; +import rx.exceptions.UnsubscribeFailedException; +import rx.internal.util.RxJavaPluginUtils; /** * {@code SafeSubscriber} is a wrapper around {@code Subscriber} that ensures that the {@code Subscriber} @@ -83,11 +85,17 @@ public void onCompleted() { // we handle here instead of another method so we don't add stacks to the frame // which can prevent it from being able to handle StackOverflow Exceptions.throwIfFatal(e); - // handle errors if the onCompleted implementation fails, not just if the Observable fails - _onError(e); + RxJavaPluginUtils.handleException(e); + throw new OnCompletedFailedException(e.getMessage(), e); } finally { - // auto-unsubscribe - unsubscribe(); + try { + // Similarly to onError if failure occurs in unsubscribe then Rx contract is broken + // and we throw an UnsubscribeFailureException. + unsubscribe(); + } catch (Throwable e) { + RxJavaPluginUtils.handleException(e); + throw new UnsubscribeFailedException(e.getMessage(), e); + } } } } @@ -145,11 +153,7 @@ public void onNext(T args) { * @see the report of this bug */ protected void _onError(Throwable e) { - try { - RxJavaPlugins.getInstance().getErrorHandler().handleError(e); - } catch (Throwable pluginException) { - handlePluginException(pluginException); - } + RxJavaPluginUtils.handleException(e); try { actual.onError(e); } catch (Throwable e2) { @@ -168,11 +172,7 @@ protected void _onError(Throwable e) { try { unsubscribe(); } catch (Throwable unsubscribeException) { - try { - RxJavaPlugins.getInstance().getErrorHandler().handleError(unsubscribeException); - } catch (Throwable pluginException) { - handlePluginException(pluginException); - } + RxJavaPluginUtils.handleException(unsubscribeException); throw new RuntimeException("Observer.onError not implemented and error while unsubscribing.", new CompositeException(Arrays.asList(e, unsubscribeException))); } throw (OnErrorNotImplementedException) e2; @@ -182,19 +182,11 @@ protected void _onError(Throwable e) { * * https://github.com/ReactiveX/RxJava/issues/198 */ - try { - RxJavaPlugins.getInstance().getErrorHandler().handleError(e2); - } catch (Throwable pluginException) { - handlePluginException(pluginException); - } + RxJavaPluginUtils.handleException(e2); try { unsubscribe(); } catch (Throwable unsubscribeException) { - try { - RxJavaPlugins.getInstance().getErrorHandler().handleError(unsubscribeException); - } catch (Throwable pluginException) { - handlePluginException(pluginException); - } + RxJavaPluginUtils.handleException(unsubscribeException); throw new OnErrorFailedException("Error occurred when trying to propagate error to Observer.onError and during unsubscription.", new CompositeException(Arrays.asList(e, e2, unsubscribeException))); } @@ -205,25 +197,11 @@ protected void _onError(Throwable e) { try { unsubscribe(); } catch (RuntimeException unsubscribeException) { - try { - RxJavaPlugins.getInstance().getErrorHandler().handleError(unsubscribeException); - } catch (Throwable pluginException) { - handlePluginException(pluginException); - } + RxJavaPluginUtils.handleException(unsubscribeException); throw new OnErrorFailedException(unsubscribeException); } } - private void handlePluginException(Throwable pluginException) { - /* - * We don't want errors from the plugin to affect normal flow. - * Since the plugin should never throw this is a safety net - * and will complain loudly to System.err so it gets fixed. - */ - System.err.println("RxJavaErrorHandler threw an Exception. It shouldn't. => " + pluginException.getMessage()); - pluginException.printStackTrace(); - } - /** * Returns the {@link Subscriber} underlying this {@code SafeSubscriber}. * diff --git a/src/test/java/rx/observers/SafeObserverTest.java b/src/test/java/rx/observers/SafeObserverTest.java index 1083e995c7..7924bb4026 100644 --- a/src/test/java/rx/observers/SafeObserverTest.java +++ b/src/test/java/rx/observers/SafeObserverTest.java @@ -22,6 +22,7 @@ import org.junit.Test; +import junit.framework.Assert; import rx.Subscriber; import rx.exceptions.*; import rx.functions.Action0; @@ -68,19 +69,6 @@ public void onCompletedFailure() { } } - @Test - public void onCompletedFailureSafe() { - AtomicReference onError = new AtomicReference(); - try { - new SafeSubscriber(OBSERVER_ONCOMPLETED_FAIL(onError)).onCompleted(); - assertNotNull(onError.get()); - assertTrue(onError.get() instanceof SafeObserverTestException); - assertEquals("onCompletedFail", onError.get().getMessage()); - } catch (Exception e) { - fail("expects exception to be passed to onError"); - } - } - @Test public void onErrorFailure() { try { @@ -184,8 +172,8 @@ public void call() { e.printStackTrace(); assertTrue(o.isUnsubscribed()); - - assertTrue(e instanceof SafeObserverTestException); + assertTrue(e instanceof UnsubscribeFailedException); + assertTrue(e.getCause() instanceof SafeObserverTestException); assertEquals("failure from unsubscribe", e.getMessage()); // expected since onError fails so SafeObserver can't help } @@ -475,9 +463,12 @@ public void onCompleted() { } }); - s.onCompleted(); - - assertTrue("Error not received", error.get() instanceof TestException); + try { + s.onCompleted(); + Assert.fail(); + } catch (OnCompletedFailedException e) { + assertNull(error.get()); + } } @Test diff --git a/src/test/java/rx/observers/SafeSubscriberTest.java b/src/test/java/rx/observers/SafeSubscriberTest.java index 85c2d7b07f..5ce37cdea4 100644 --- a/src/test/java/rx/observers/SafeSubscriberTest.java +++ b/src/test/java/rx/observers/SafeSubscriberTest.java @@ -15,15 +15,25 @@ */ package rx.observers; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import java.lang.reflect.Method; +import java.util.concurrent.atomic.AtomicInteger; -import org.junit.*; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; -import rx.exceptions.*; +import rx.exceptions.OnCompletedFailedException; +import rx.exceptions.OnErrorFailedException; +import rx.exceptions.OnErrorNotImplementedException; +import rx.exceptions.TestException; +import rx.exceptions.UnsubscribeFailedException; import rx.functions.Action0; -import rx.plugins.*; +import rx.plugins.RxJavaErrorHandler; +import rx.plugins.RxJavaPlugins; import rx.subscriptions.Subscriptions; public class SafeSubscriberTest { @@ -51,10 +61,12 @@ public void onCompleted() { } }; SafeSubscriber safe = new SafeSubscriber(ts); - - safe.onCompleted(); - - assertTrue(safe.isUnsubscribed()); + try { + safe.onCompleted(); + Assert.fail(); + } catch (OnCompletedFailedException e) { + assertTrue(safe.isUnsubscribed()); + } } @Test @@ -76,7 +88,7 @@ public void onCompleted() { assertTrue(safe.isUnsubscribed()); } - @Test + @Test(expected=OnCompletedFailedException.class) public void testPluginException() { RxJavaPlugins.getInstance().registerErrorHandler(new RxJavaErrorHandler() { @Override @@ -227,4 +239,81 @@ public void call() { safe.onError(new TestException()); } + + @Test + public void testPluginErrorHandlerReceivesExceptionWhenUnsubscribeAfterCompletionThrows() { + final AtomicInteger calls = new AtomicInteger(); + RxJavaPlugins.getInstance().registerErrorHandler(new RxJavaErrorHandler() { + @Override + public void handleError(Throwable e) { + calls.incrementAndGet(); + } + }); + + final AtomicInteger errors = new AtomicInteger(); + TestSubscriber ts = new TestSubscriber() { + @Override + public void onError(Throwable e) { + errors.incrementAndGet(); + } + }; + final RuntimeException ex = new RuntimeException(); + SafeSubscriber safe = new SafeSubscriber(ts); + safe.add(Subscriptions.create(new Action0() { + @Override + public void call() { + throw ex; + } + })); + + try { + safe.onCompleted(); + Assert.fail(); + } catch(UnsubscribeFailedException e) { + assertEquals(1, (int) calls.get()); + assertEquals(0, (int) errors.get()); + } + } + + @Test + public void testPluginErrorHandlerReceivesExceptionFromFailingUnsubscribeAfterCompletionThrows() { + final AtomicInteger calls = new AtomicInteger(); + RxJavaPlugins.getInstance().registerErrorHandler(new RxJavaErrorHandler() { + @Override + public void handleError(Throwable e) { + calls.incrementAndGet(); + } + }); + + final AtomicInteger errors = new AtomicInteger(); + TestSubscriber ts = new TestSubscriber() { + + @Override + public void onCompleted() { + throw new RuntimeException(); + } + + @Override + public void onError(Throwable e) { + errors.incrementAndGet(); + } + }; + SafeSubscriber safe = new SafeSubscriber(ts); + safe.add(Subscriptions.create(new Action0() { + @Override + public void call() { + throw new RuntimeException(); + } + })); + + try { + safe.onCompleted(); + Assert.fail(); + } catch(UnsubscribeFailedException e) { + assertEquals(2, (int) calls.get()); + assertEquals(0, (int) errors.get()); + } + } + + } From b757ece11caada0b28c774b232664ccc4d73bff0 Mon Sep 17 00:00:00 2001 From: akarnokd Date: Sat, 22 Aug 2015 12:35:40 +0200 Subject: [PATCH 43/81] BackpressureUtils capped add/multiply methods + tests --- .../internal/operators/BackpressureUtils.java | 43 ++++++++++++++----- .../operators/BackpressureUtilsTest.java | 39 +++++++++++++++++ 2 files changed, 72 insertions(+), 10 deletions(-) create mode 100644 src/test/java/rx/internal/operators/BackpressureUtilsTest.java diff --git a/src/main/java/rx/internal/operators/BackpressureUtils.java b/src/main/java/rx/internal/operators/BackpressureUtils.java index 505b248553..937f186535 100644 --- a/src/main/java/rx/internal/operators/BackpressureUtils.java +++ b/src/main/java/rx/internal/operators/BackpressureUtils.java @@ -44,11 +44,7 @@ public static long getAndAddRequest(AtomicLongFieldUpdater requested, T o // add n to field but check for overflow while (true) { long current = requested.get(object); - long next = current + n; - // check for overflow - if (next < 0) { - next = Long.MAX_VALUE; - } + long next = addCap(current, n); if (requested.compareAndSet(object, current, next)) { return current; } @@ -70,14 +66,41 @@ public static long getAndAddRequest(AtomicLong requested, long n) { // add n to field but check for overflow while (true) { long current = requested.get(); - long next = current + n; - // check for overflow - if (next < 0) { - next = Long.MAX_VALUE; - } + long next = addCap(current, n); if (requested.compareAndSet(current, next)) { return current; } } } + + /** + * Multiplies two positive longs and caps the result at Long.MAX_VALUE. + * @param a the first value + * @param b the second value + * @return the capped product of a and b + */ + public static long multiplyCap(long a, long b) { + long u = a * b; + if (((a | b) >>> 31) != 0) { + if (b != 0L && (u / b != a)) { + u = Long.MAX_VALUE; + } + } + return u; + } + + /** + * Adds two positive longs and caps the result at Long.MAX_VALUE. + * @param a the first value + * @param b the second value + * @return the capped sum of a and b + */ + public static long addCap(long a, long b) { + long u = a + b; + if (u < 0L) { + u = Long.MAX_VALUE; + } + return u; + } + } diff --git a/src/test/java/rx/internal/operators/BackpressureUtilsTest.java b/src/test/java/rx/internal/operators/BackpressureUtilsTest.java new file mode 100644 index 0000000000..0bc7f542bf --- /dev/null +++ b/src/test/java/rx/internal/operators/BackpressureUtilsTest.java @@ -0,0 +1,39 @@ +/** + * Copyright 2015 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.internal.operators; + +import org.junit.Test; +import static org.junit.Assert.*; + +public class BackpressureUtilsTest { + @Test + public void testAddCap() { + assertEquals(2L, BackpressureUtils.addCap(1, 1)); + assertEquals(Long.MAX_VALUE, BackpressureUtils.addCap(1, Long.MAX_VALUE - 1)); + assertEquals(Long.MAX_VALUE, BackpressureUtils.addCap(1, Long.MAX_VALUE)); + assertEquals(Long.MAX_VALUE, BackpressureUtils.addCap(Long.MAX_VALUE - 1, Long.MAX_VALUE - 1)); + assertEquals(Long.MAX_VALUE, BackpressureUtils.addCap(Long.MAX_VALUE, Long.MAX_VALUE)); + } + + @Test + public void testMultiplyCap() { + assertEquals(6, BackpressureUtils.multiplyCap(2, 3)); + assertEquals(Long.MAX_VALUE, BackpressureUtils.multiplyCap(2, Long.MAX_VALUE)); + assertEquals(Long.MAX_VALUE, BackpressureUtils.multiplyCap(Long.MAX_VALUE, Long.MAX_VALUE)); + assertEquals(Long.MAX_VALUE, BackpressureUtils.multiplyCap(1L << 32, 1L << 32)); + + } +} From 49692e4a530c668ebbfae76ec607ba0f6cda65a4 Mon Sep 17 00:00:00 2001 From: akarnokd Date: Thu, 20 Aug 2015 19:19:32 +0200 Subject: [PATCH 44/81] Fixed negative request due to unsubscription of a large requester --- .../rx/internal/operators/OperatorReplay.java | 2 +- .../operators/OperatorReplayTest.java | 25 +++++++++++++++++++ 2 files changed, 26 insertions(+), 1 deletion(-) diff --git a/src/main/java/rx/internal/operators/OperatorReplay.java b/src/main/java/rx/internal/operators/OperatorReplay.java index e1bf7aa352..b7b52aded3 100644 --- a/src/main/java/rx/internal/operators/OperatorReplay.java +++ b/src/main/java/rx/internal/operators/OperatorReplay.java @@ -501,7 +501,7 @@ void manageRequests() { InnerProducer[] a = producers.get(); long ri = maxChildRequested; - long maxTotalRequests = 0; + long maxTotalRequests = ri; for (InnerProducer rp : a) { maxTotalRequests = Math.max(maxTotalRequests, rp.totalRequested.get()); diff --git a/src/test/java/rx/internal/operators/OperatorReplayTest.java b/src/test/java/rx/internal/operators/OperatorReplayTest.java index 046803b082..c0ec384d84 100644 --- a/src/test/java/rx/internal/operators/OperatorReplayTest.java +++ b/src/test/java/rx/internal/operators/OperatorReplayTest.java @@ -1120,4 +1120,29 @@ public void onNext(Integer t) { ts.assertNotCompleted(); ts.assertError(TestException.class); } + + @Test + public void unboundedLeavesEarly() { + PublishSubject source = PublishSubject.create(); + + final List requests = new ArrayList(); + + Observable out = source + .doOnRequest(new Action1() { + @Override + public void call(Long t) { + requests.add(t); + } + }).replay().autoConnect(); + + TestSubscriber ts1 = TestSubscriber.create(5); + TestSubscriber ts2 = TestSubscriber.create(10); + + out.subscribe(ts1); + out.subscribe(ts2); + ts2.unsubscribe(); + + Assert.assertEquals(Arrays.asList(5L, 5L), requests); + } + } \ No newline at end of file From 6d4f25c8cc66ae54038778452e9078e6f6ae377a Mon Sep 17 00:00:00 2001 From: akarnokd Date: Mon, 24 Aug 2015 19:17:24 +0200 Subject: [PATCH 45/81] MapNotification producer NPE fix --- .../operators/OperatorMapNotification.java | 96 +++++++++++-------- .../OperatorMapNotificationTest.java | 55 +++++++++++ 2 files changed, 109 insertions(+), 42 deletions(-) create mode 100644 src/test/java/rx/internal/operators/OperatorMapNotificationTest.java diff --git a/src/main/java/rx/internal/operators/OperatorMapNotification.java b/src/main/java/rx/internal/operators/OperatorMapNotification.java index be363663fb..bb92f2c077 100644 --- a/src/main/java/rx/internal/operators/OperatorMapNotification.java +++ b/src/main/java/rx/internal/operators/OperatorMapNotification.java @@ -19,16 +19,12 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicLong; +import rx.*; import rx.Observable.Operator; -import rx.Producer; -import rx.Subscriber; -import rx.Subscription; -import rx.exceptions.MissingBackpressureException; -import rx.exceptions.OnErrorThrowable; -import rx.functions.Func0; -import rx.functions.Func1; -import rx.internal.util.unsafe.SpscArrayQueue; -import rx.internal.util.unsafe.UnsafeAccess; +import rx.exceptions.*; +import rx.functions.*; +import rx.internal.producers.ProducerArbiter; +import rx.internal.util.unsafe.*; /** * Applies a function of your choosing to every item emitted by an {@code Observable}, and emits the results of @@ -50,44 +46,60 @@ public OperatorMapNotification(Func1 onNext, Func1 call(final Subscriber o) { - Subscriber subscriber = new Subscriber() { - SingleEmitter emitter; - @Override - public void setProducer(Producer producer) { - emitter = new SingleEmitter(o, producer, this); - o.setProducer(emitter); - } - - @Override - public void onCompleted() { - try { - emitter.offerAndComplete(onCompleted.call()); - } catch (Throwable e) { - o.onError(e); - } - } + final ProducerArbiter pa = new ProducerArbiter(); + + MapNotificationSubscriber subscriber = new MapNotificationSubscriber(pa, o); + o.add(subscriber); + subscriber.init(); + return subscriber; + } + + final class MapNotificationSubscriber extends Subscriber { + private final Subscriber o; + private final ProducerArbiter pa; + final SingleEmitter emitter; + + private MapNotificationSubscriber(ProducerArbiter pa, Subscriber o) { + this.pa = pa; + this.o = o; + this.emitter = new SingleEmitter(o, pa, this); + } + + void init() { + o.setProducer(emitter); + } - @Override - public void onError(Throwable e) { - try { - emitter.offerAndComplete(onError.call(e)); - } catch (Throwable e2) { - o.onError(e); - } + @Override + public void setProducer(Producer producer) { + pa.setProducer(producer); + } + + @Override + public void onCompleted() { + try { + emitter.offerAndComplete(onCompleted.call()); + } catch (Throwable e) { + o.onError(e); } + } - @Override - public void onNext(T t) { - try { - emitter.offer(onNext.call(t)); - } catch (Throwable e) { - o.onError(OnErrorThrowable.addValueAsLastCause(e, t)); - } + @Override + public void onError(Throwable e) { + try { + emitter.offerAndComplete(onError.call(e)); + } catch (Throwable e2) { + o.onError(e); } + } - }; - o.add(subscriber); - return subscriber; + @Override + public void onNext(T t) { + try { + emitter.offer(onNext.call(t)); + } catch (Throwable e) { + o.onError(OnErrorThrowable.addValueAsLastCause(e, t)); + } + } } static final class SingleEmitter extends AtomicLong implements Producer, Subscription { /** */ diff --git a/src/test/java/rx/internal/operators/OperatorMapNotificationTest.java b/src/test/java/rx/internal/operators/OperatorMapNotificationTest.java new file mode 100644 index 0000000000..2f1e603337 --- /dev/null +++ b/src/test/java/rx/internal/operators/OperatorMapNotificationTest.java @@ -0,0 +1,55 @@ +/** + * Copyright 2014 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.internal.operators; + +import org.junit.Test; + +import rx.Observable; +import rx.functions.*; +import rx.observers.TestSubscriber; + +public class OperatorMapNotificationTest { + @Test + public void testJust() { + TestSubscriber ts = TestSubscriber.create(); + Observable.just(1) + .flatMap( + new Func1>() { + @Override + public Observable call(Integer item) { + return Observable.just((Object)(item + 1)); + } + }, + new Func1>() { + @Override + public Observable call(Throwable e) { + return Observable.error(e); + } + }, + new Func0>() { + @Override + public Observable call() { + return Observable.never(); + } + } + ).subscribe(ts); + + ts.assertNoErrors(); + ts.assertNotCompleted(); + ts.assertValue(2); + } +} From 5804bede0725d1a0056a9528a87a04c743e732e0 Mon Sep 17 00:00:00 2001 From: akarnokd Date: Fri, 21 Aug 2015 14:05:44 +0200 Subject: [PATCH 46/81] Scan backpressure and first emission fix --- .../rx/internal/operators/OperatorScan.java | 340 ++++++++++++++---- .../internal/operators/OperatorScanTest.java | 69 +++- 2 files changed, 321 insertions(+), 88 deletions(-) diff --git a/src/main/java/rx/internal/operators/OperatorScan.java b/src/main/java/rx/internal/operators/OperatorScan.java index 788842100d..1cbdb53d54 100644 --- a/src/main/java/rx/internal/operators/OperatorScan.java +++ b/src/main/java/rx/internal/operators/OperatorScan.java @@ -15,15 +15,14 @@ */ package rx.internal.operators; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.Queue; +import rx.*; import rx.Observable.Operator; -import rx.Producer; -import rx.Subscriber; -import rx.exceptions.Exceptions; -import rx.exceptions.OnErrorThrowable; -import rx.functions.Func0; -import rx.functions.Func2; +import rx.exceptions.*; +import rx.functions.*; +import rx.internal.util.atomic.SpscLinkedAtomicQueue; +import rx.internal.util.unsafe.*; /** * Returns an Observable that applies a function to the first item emitted by a source Observable, then feeds @@ -87,87 +86,290 @@ public OperatorScan(final Func2 accumulator) { @Override public Subscriber call(final Subscriber child) { - return new Subscriber(child) { - private final R initialValue = initialValueFactory.call(); + final R initialValue = initialValueFactory.call(); + + if (initialValue == NO_INITIAL_VALUE) { + return new Subscriber(child) { + boolean once; + R value; + @SuppressWarnings("unchecked") + @Override + public void onNext(T t) { + R v; + if (!once) { + once = true; + v = (R)t; + } else { + v = value; + try { + v = accumulator.call(v, t); + } catch (Throwable e) { + Exceptions.throwIfFatal(e); + child.onError(OnErrorThrowable.addValueAsLastCause(e, t)); + return; + } + } + value = v; + child.onNext(v); + } + @Override + public void onError(Throwable e) { + child.onError(e); + } + @Override + public void onCompleted() { + child.onCompleted(); + } + }; + } + + final InitialProducer ip = new InitialProducer(initialValue, child); + + Subscriber parent = new Subscriber() { private R value = initialValue; - boolean initialized = false; - @SuppressWarnings("unchecked") @Override public void onNext(T currentValue) { - emitInitialValueIfNeeded(child); - - if (this.value == NO_INITIAL_VALUE) { - // if there is NO_INITIAL_VALUE then we know it is type T for both so cast T to R - this.value = (R) currentValue; - } else { - try { - this.value = accumulator.call(this.value, currentValue); - } catch (Throwable e) { - Exceptions.throwIfFatal(e); - child.onError(OnErrorThrowable.addValueAsLastCause(e, currentValue)); - return; - } + R v = value; + try { + v = accumulator.call(v, currentValue); + } catch (Throwable e) { + Exceptions.throwIfFatal(e); + onError(OnErrorThrowable.addValueAsLastCause(e, currentValue)); + return; } - child.onNext(this.value); + value = v; + ip.onNext(v); } @Override public void onError(Throwable e) { - child.onError(e); + ip.onError(e); } @Override public void onCompleted() { - emitInitialValueIfNeeded(child); - child.onCompleted(); - } - - private void emitInitialValueIfNeeded(final Subscriber child) { - if (!initialized) { - initialized = true; - // we emit first time through if we have an initial value - if (initialValue != NO_INITIAL_VALUE) { - child.onNext(initialValue); - } - } + ip.onCompleted(); } - /** - * We want to adjust the requested value by subtracting 1 if we have an initial value - */ @Override public void setProducer(final Producer producer) { - child.setProducer(new Producer() { - - final AtomicBoolean once = new AtomicBoolean(); - - final AtomicBoolean excessive = new AtomicBoolean(); - - @Override - public void request(long n) { - if (once.compareAndSet(false, true)) { - if (initialValue == NO_INITIAL_VALUE || n == Long.MAX_VALUE) { - producer.request(n); - } else if (n == 1) { - excessive.set(true); - producer.request(1); // request at least 1 - } else { - // n != Long.MAX_VALUE && n != 1 - producer.request(n - 1); - } - } else { - // pass-thru after first time - if (n > 1 // avoid to request 0 - && excessive.compareAndSet(true, false) && n != Long.MAX_VALUE) { - producer.request(n - 1); - } else { - producer.request(n); - } + ip.setProducer(producer); + } + }; + + child.add(parent); + child.setProducer(ip); + return parent; + } + + static final class InitialProducer implements Producer, Observer { + final Subscriber child; + final Queue queue; + + boolean emitting; + /** Missed a terminal event. */ + boolean missed; + /** Missed a request. */ + long missedRequested; + /** Missed a producer. */ + Producer missedProducer; + /** The current requested amount. */ + long requested; + /** The current producer. */ + Producer producer; + + volatile boolean done; + Throwable error; + + public InitialProducer(R initialValue, Subscriber child) { + this.child = child; + Queue q; + // TODO switch to the linked-array based queue once available + if (UnsafeAccess.isUnsafeAvailable()) { + q = new SpscLinkedQueue(); // new SpscUnboundedArrayQueue(8); + } else { + q = new SpscLinkedAtomicQueue(); // new SpscUnboundedAtomicArrayQueue(8); + } + this.queue = q; + q.offer(initialValue); + } + + @Override + public void request(long n) { + if (n < 0L) { + throw new IllegalArgumentException("n >= required but it was " + n); + } else + if (n != 0L) { + synchronized (this) { + if (emitting) { + long mr = missedRequested; + long mu = mr + n; + if (mu < 0L) { + mu = Long.MAX_VALUE; } + missedRequested = mu; + return; } - }); + emitting = true; + } + + long r = requested; + long u = r + n; + if (u < 0L) { + u = Long.MAX_VALUE; + } + requested = u; + + Producer p = producer; + if (p != null) { + p.request(n); + } + + emitLoop(); } - }; + } + + @Override + public void onNext(R t) { + queue.offer(NotificationLite.instance().next(t)); + emit(); + } + + boolean checkTerminated(boolean d, boolean empty, Subscriber child) { + if (child.isUnsubscribed()) { + return true; + } + if (d) { + Throwable err = error; + if (err != null) { + child.onError(err); + return true; + } else + if (empty) { + child.onCompleted(); + return true; + } + } + return false; + } + + @Override + public void onError(Throwable e) { + error = e; + done = true; + emit(); + } + + @Override + public void onCompleted() { + done = true; + emit(); + } + + public void setProducer(Producer p) { + if (p == null) { + throw new NullPointerException(); + } + synchronized (this) { + if (emitting) { + missedProducer = p; + return; + } + emitting = true; + } + producer = p; + long r = requested; + if (r != 0L) { + p.request(r); + } + emitLoop(); + } + + void emit() { + synchronized (this) { + if (emitting) { + missed = true; + return; + } + emitting = true; + } + emitLoop(); + } + + void emitLoop() { + final Subscriber child = this.child; + final Queue queue = this.queue; + final NotificationLite nl = NotificationLite.instance(); + long r = requested; + for (;;) { + boolean max = r == Long.MAX_VALUE; + boolean d = done; + boolean empty = queue.isEmpty(); + if (checkTerminated(d, empty, child)) { + return; + } + while (r != 0L) { + d = done; + Object o = queue.poll(); + empty = o == null; + if (checkTerminated(d, empty, child)) { + return; + } + if (empty) { + break; + } + R v = nl.getValue(o); + try { + child.onNext(v); + } catch (Throwable e) { + Exceptions.throwIfFatal(e); + child.onError(OnErrorThrowable.addValueAsLastCause(e, v)); + return; + } + if (!max) { + r--; + } + } + if (!max) { + requested = r; + } + + Producer p; + long mr; + synchronized (this) { + p = missedProducer; + mr = missedRequested; + if (!missed && p == null && mr == 0L) { + emitting = false; + return; + } + missed = false; + missedProducer = null; + missedRequested = 0L; + } + + if (mr != 0L && !max) { + long u = r + mr; + if (u < 0L) { + u = Long.MAX_VALUE; + } + requested = u; + r = u; + } + + if (p != null) { + producer = p; + if (r != 0L) { + p.request(r); + } + } else { + p = producer; + if (p != null && mr != 0L) { + p.request(mr); + } + } + } + } } } diff --git a/src/test/java/rx/internal/operators/OperatorScanTest.java b/src/test/java/rx/internal/operators/OperatorScanTest.java index e05d4d9bb1..ac7772753f 100644 --- a/src/test/java/rx/internal/operators/OperatorScanTest.java +++ b/src/test/java/rx/internal/operators/OperatorScanTest.java @@ -15,33 +15,23 @@ */ package rx.internal.operators; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyString; +import static org.junit.Assert.*; +import static org.mockito.Matchers.*; import static org.mockito.Mockito.*; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; +import java.util.*; +import java.util.concurrent.atomic.*; -import org.junit.Before; -import org.junit.Test; +import org.junit.*; import org.mockito.MockitoAnnotations; +import rx.*; import rx.Observable; import rx.Observer; -import rx.Producer; -import rx.Subscriber; -import rx.functions.Action2; -import rx.functions.Func0; -import rx.functions.Func1; -import rx.functions.Func2; +import rx.functions.*; +import rx.observables.AbstractOnSubscribe; import rx.observers.TestSubscriber; +import rx.subjects.PublishSubject; public class OperatorScanTest { @@ -360,4 +350,45 @@ public void onNext(Integer integer) { verify(producer.get(), never()).request(0); verify(producer.get(), times(2)).request(1); } + + @Test + public void testInitialValueEmittedNoProducer() { + PublishSubject source = PublishSubject.create(); + + TestSubscriber ts = TestSubscriber.create(); + + source.scan(0, new Func2() { + @Override + public Integer call(Integer t1, Integer t2) { + return t1 + t2; + } + }).subscribe(ts); + + ts.assertNoErrors(); + ts.assertNotCompleted(); + ts.assertValue(0); + } + + @Test + public void testInitialValueEmittedWithProducer() { + Observable source = new AbstractOnSubscribe() { + @Override + protected void next(rx.observables.AbstractOnSubscribe.SubscriptionState state) { + state.stop(); + } + }.toObservable(); + + TestSubscriber ts = TestSubscriber.create(); + + source.scan(0, new Func2() { + @Override + public Integer call(Integer t1, Integer t2) { + return t1 + t2; + } + }).subscribe(ts); + + ts.assertNoErrors(); + ts.assertNotCompleted(); + ts.assertValue(0); + } } From e5ec3f049067fcb0744d1d317afcef841e45a28b Mon Sep 17 00:00:00 2001 From: wrightm Date: Thu, 27 Aug 2015 22:05:22 +0100 Subject: [PATCH 47/81] Fix to Notification equals method. --- src/main/java/rx/Notification.java | 5 ++ src/test/java/rx/NotificationTest.java | 64 ++++++++++++++++++++++++++ 2 files changed, 69 insertions(+) create mode 100644 src/test/java/rx/NotificationTest.java diff --git a/src/main/java/rx/Notification.java b/src/main/java/rx/Notification.java index 17a23d1031..b708b58766 100644 --- a/src/main/java/rx/Notification.java +++ b/src/main/java/rx/Notification.java @@ -202,6 +202,11 @@ public boolean equals(Object obj) { return false; if (hasThrowable() && !getThrowable().equals(notification.getThrowable())) return false; + if(!hasValue() && !hasThrowable() && notification.hasValue()) + return false; + if(!hasValue() && !hasThrowable() && notification.hasThrowable()) + return false; + return true; } } diff --git a/src/test/java/rx/NotificationTest.java b/src/test/java/rx/NotificationTest.java new file mode 100644 index 0000000000..cf33fb991a --- /dev/null +++ b/src/test/java/rx/NotificationTest.java @@ -0,0 +1,64 @@ +package rx; + +import org.junit.Assert; +import org.junit.Test; + +public class NotificationTest { + + @Test + public void testOnNextIntegerNotificationDoesNotEqualNullNotification(){ + final Notification integerNotification = Notification.createOnNext(1); + final Notification nullNotification = Notification.createOnNext(null); + Assert.assertFalse(integerNotification.equals(nullNotification)); + } + + @Test + public void testOnNextNullNotificationDoesNotEqualIntegerNotification(){ + final Notification integerNotification = Notification.createOnNext(1); + final Notification nullNotification = Notification.createOnNext(null); + Assert.assertFalse(nullNotification.equals(integerNotification)); + } + + @Test + public void testOnNextIntegerNotificationsWhenEqual(){ + final Notification integerNotification = Notification.createOnNext(1); + final Notification integerNotification2 = Notification.createOnNext(1); + Assert.assertTrue(integerNotification.equals(integerNotification2)); + } + + @Test + public void testOnNextIntegerNotificationsWhenNotEqual(){ + final Notification integerNotification = Notification.createOnNext(1); + final Notification integerNotification2 = Notification.createOnNext(2); + Assert.assertFalse(integerNotification.equals(integerNotification2)); + } + + @Test + public void testOnErrorIntegerNotificationDoesNotEqualNullNotification(){ + final Notification integerNotification = Notification.createOnError(new Exception()); + final Notification nullNotification = Notification.createOnError(null); + Assert.assertFalse(integerNotification.equals(nullNotification)); + } + + @Test + public void testOnErrorNullNotificationDoesNotEqualIntegerNotification(){ + final Notification integerNotification = Notification.createOnError(new Exception()); + final Notification nullNotification = Notification.createOnError(null); + Assert.assertFalse(nullNotification.equals(integerNotification)); + } + + @Test + public void testOnErrorIntegerNotificationsWhenEqual(){ + final Exception exception = new Exception(); + final Notification onErrorNotification = Notification.createOnError(exception); + final Notification onErrorNotification2 = Notification.createOnError(exception); + Assert.assertTrue(onErrorNotification.equals(onErrorNotification2)); + } + + @Test + public void testOnErrorIntegerNotificationWhenNotEqual(){ + final Notification onErrorNotification = Notification.createOnError(new Exception()); + final Notification onErrorNotification2 = Notification.createOnError(new Exception()); + Assert.assertFalse(onErrorNotification.equals(onErrorNotification2)); + } +} From 619abfbaa016ab1b6592afc777ee5ceeba0ceb67 Mon Sep 17 00:00:00 2001 From: akarnokd Date: Tue, 25 Aug 2015 00:10:29 +0200 Subject: [PATCH 48/81] Refactored exception reporting of most operators. --- src/main/java/rx/exceptions/Exceptions.java | 29 +++++++++++++-- .../operators/OnSubscribeCombineLatest.java | 4 +- .../internal/operators/OnSubscribeDefer.java | 3 +- ...ubscribeDelaySubscriptionWithSelector.java | 3 +- .../operators/OnSubscribeGroupJoin.java | 25 +++++-------- .../internal/operators/OnSubscribeJoin.java | 19 ++++------ .../operators/OnSubscribeTimerOnce.java | 3 +- .../OnSubscribeTimerPeriodically.java | 5 ++- .../OnSubscribeToObservableFuture.java | 3 +- .../internal/operators/OnSubscribeUsing.java | 6 ++- .../rx/internal/operators/OperatorAll.java | 4 +- .../rx/internal/operators/OperatorAny.java | 7 +--- .../OperatorBufferWithSingleObservable.java | 7 ++-- .../operators/OperatorBufferWithSize.java | 5 ++- .../OperatorBufferWithStartEndObservable.java | 5 ++- .../operators/OperatorBufferWithTime.java | 9 +++-- .../rx/internal/operators/OperatorCast.java | 4 +- .../OperatorDebounceWithSelector.java | 3 +- .../operators/OperatorDebounceWithTime.java | 5 ++- .../operators/OperatorDelayWithSelector.java | 3 +- .../internal/operators/OperatorDoOnEach.java | 10 ++--- .../rx/internal/operators/OperatorFilter.java | 4 +- .../internal/operators/OperatorGroupBy.java | 6 +-- .../rx/internal/operators/OperatorMap.java | 4 +- .../operators/OperatorMapNotification.java | 6 +-- .../internal/operators/OperatorMapPair.java | 4 +- .../OperatorOnErrorResumeNextViaFunction.java | 2 +- .../operators/OperatorOnErrorReturn.java | 1 + .../internal/operators/OperatorPublish.java | 4 +- .../rx/internal/operators/OperatorReplay.java | 3 +- .../operators/OperatorSampleWithTime.java | 3 +- .../rx/internal/operators/OperatorSkip.java | 6 +-- .../operators/OperatorTakeLastOne.java | 3 +- .../operators/OperatorTakeUntilPredicate.java | 6 +-- .../internal/operators/OperatorTakeWhile.java | 9 ++--- .../OperatorTimeoutWithSelector.java | 6 +-- .../operators/OperatorToObservableList.java | 3 +- .../OperatorToObservableSortedList.java | 3 +- .../operators/OperatorWithLatestFrom.java | 3 +- .../rx/internal/operators/OperatorZip.java | 5 +-- .../operators/OperatorZipIterable.java | 6 ++- .../operators/TakeLastQueueProducer.java | 3 +- .../producers/ProducerObserverArbiter.java | 4 +- .../rx/internal/producers/QueuedProducer.java | 4 +- .../producers/QueuedValueProducer.java | 4 +- .../producers/SingleDelayedProducer.java | 4 +- .../rx/internal/producers/SingleProducer.java | 3 +- .../operators/OperatorFilterTest.java | 37 +++++++++++++++---- 48 files changed, 167 insertions(+), 141 deletions(-) diff --git a/src/main/java/rx/exceptions/Exceptions.java b/src/main/java/rx/exceptions/Exceptions.java index b8907bf436..1b29838637 100644 --- a/src/main/java/rx/exceptions/Exceptions.java +++ b/src/main/java/rx/exceptions/Exceptions.java @@ -15,10 +15,9 @@ */ package rx.exceptions; -import java.util.HashSet; -import java.util.List; -import java.util.Set; +import java.util.*; +import rx.Observer; import rx.annotations.Experimental; /** @@ -178,4 +177,28 @@ public static void throwIfAny(List exceptions) { "Multiple exceptions", exceptions); } } + + /** + * Forwards a fatal exception or reports it along with the value + * caused it to the given Observer. + * @param t the exception + * @param o the observer to report to + * @param value the value that caused the exception + */ + @Experimental + public static void throwOrReport(Throwable t, Observer o, Object value) { + Exceptions.throwIfFatal(t); + o.onError(OnErrorThrowable.addValueAsLastCause(t, value)); + } + /** + * Forwards a fatal exception or reports it to the given Observer. + * @param t the exception + * @param o the observer to report to + * @param value the value that caused the exception + */ + @Experimental + public static void throwOrReport(Throwable t, Observer o) { + Exceptions.throwIfFatal(t); + o.onError(t); + } } diff --git a/src/main/java/rx/internal/operators/OnSubscribeCombineLatest.java b/src/main/java/rx/internal/operators/OnSubscribeCombineLatest.java index 953895af32..54e1335205 100644 --- a/src/main/java/rx/internal/operators/OnSubscribeCombineLatest.java +++ b/src/main/java/rx/internal/operators/OnSubscribeCombineLatest.java @@ -23,9 +23,9 @@ import rx.Observable; import rx.Observable.OnSubscribe; +import rx.exceptions.*; import rx.Producer; import rx.Subscriber; -import rx.exceptions.MissingBackpressureException; import rx.functions.FuncN; import rx.internal.util.RxRingBuffer; @@ -202,7 +202,7 @@ public boolean onNext(int index, T t) { } catch (MissingBackpressureException e) { onError(e); } catch (Throwable e) { - onError(e); + Exceptions.throwOrReport(e, child); } } } diff --git a/src/main/java/rx/internal/operators/OnSubscribeDefer.java b/src/main/java/rx/internal/operators/OnSubscribeDefer.java index 23ee937145..34a060df41 100644 --- a/src/main/java/rx/internal/operators/OnSubscribeDefer.java +++ b/src/main/java/rx/internal/operators/OnSubscribeDefer.java @@ -18,6 +18,7 @@ import rx.Observable; import rx.Observable.OnSubscribe; import rx.Subscriber; +import rx.exceptions.Exceptions; import rx.functions.Func0; import rx.observers.Subscribers; @@ -44,7 +45,7 @@ public void call(final Subscriber s) { try { o = observableFactory.call(); } catch (Throwable t) { - s.onError(t); + Exceptions.throwOrReport(t, s); return; } o.unsafeSubscribe(Subscribers.wrap(s)); diff --git a/src/main/java/rx/internal/operators/OnSubscribeDelaySubscriptionWithSelector.java b/src/main/java/rx/internal/operators/OnSubscribeDelaySubscriptionWithSelector.java index b32179b3f7..8c57c44f62 100644 --- a/src/main/java/rx/internal/operators/OnSubscribeDelaySubscriptionWithSelector.java +++ b/src/main/java/rx/internal/operators/OnSubscribeDelaySubscriptionWithSelector.java @@ -17,6 +17,7 @@ import rx.*; import rx.Observable.OnSubscribe; +import rx.exceptions.Exceptions; import rx.functions.Func0; import rx.observers.Subscribers; @@ -58,7 +59,7 @@ public void onNext(U t) { }); } catch (Throwable e) { - child.onError(e); + Exceptions.throwOrReport(e, child); } } diff --git a/src/main/java/rx/internal/operators/OnSubscribeGroupJoin.java b/src/main/java/rx/internal/operators/OnSubscribeGroupJoin.java index e80b560dcd..4b7509c2d9 100644 --- a/src/main/java/rx/internal/operators/OnSubscribeGroupJoin.java +++ b/src/main/java/rx/internal/operators/OnSubscribeGroupJoin.java @@ -15,24 +15,17 @@ */ package rx.internal.operators; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; +import java.util.*; +import rx.*; import rx.Observable; import rx.Observable.OnSubscribe; import rx.Observer; -import rx.Subscriber; -import rx.Subscription; -import rx.functions.Func1; -import rx.functions.Func2; -import rx.observers.SerializedObserver; -import rx.observers.SerializedSubscriber; -import rx.subjects.PublishSubject; -import rx.subjects.Subject; -import rx.subscriptions.CompositeSubscription; -import rx.subscriptions.RefCountSubscription; +import rx.exceptions.Exceptions; +import rx.functions.*; +import rx.observers.*; +import rx.subjects.*; +import rx.subscriptions.*; /** * Corrrelates two sequences when they overlap and groups the results. @@ -192,7 +185,7 @@ public void onNext(T1 args) { } catch (Throwable t) { - onError(t); + Exceptions.throwOrReport(t, this); } } @@ -242,7 +235,7 @@ public void onNext(T2 args) { o.onNext(args); } } catch (Throwable t) { - onError(t); + Exceptions.throwOrReport(t, this); } } diff --git a/src/main/java/rx/internal/operators/OnSubscribeJoin.java b/src/main/java/rx/internal/operators/OnSubscribeJoin.java index b6edd5c366..f93437c5d0 100644 --- a/src/main/java/rx/internal/operators/OnSubscribeJoin.java +++ b/src/main/java/rx/internal/operators/OnSubscribeJoin.java @@ -15,20 +15,15 @@ */ package rx.internal.operators; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; +import java.util.*; +import rx.*; import rx.Observable; import rx.Observable.OnSubscribe; -import rx.Subscriber; -import rx.Subscription; -import rx.functions.Func1; -import rx.functions.Func2; +import rx.exceptions.Exceptions; +import rx.functions.*; import rx.observers.SerializedSubscriber; -import rx.subscriptions.CompositeSubscription; -import rx.subscriptions.SerialSubscription; +import rx.subscriptions.*; /** * Correlates the elements of two sequences based on overlapping durations. @@ -154,7 +149,7 @@ public void onNext(TLeft args) { subscriber.onNext(result); } } catch (Throwable t) { - onError(t); + Exceptions.throwOrReport(t, this); } } @@ -266,7 +261,7 @@ public void onNext(TRight args) { } } catch (Throwable t) { - onError(t); + Exceptions.throwOrReport(t, this); } } diff --git a/src/main/java/rx/internal/operators/OnSubscribeTimerOnce.java b/src/main/java/rx/internal/operators/OnSubscribeTimerOnce.java index cf31ae6ca8..2b618ac21f 100644 --- a/src/main/java/rx/internal/operators/OnSubscribeTimerOnce.java +++ b/src/main/java/rx/internal/operators/OnSubscribeTimerOnce.java @@ -19,6 +19,7 @@ import rx.Observable.OnSubscribe; import rx.Scheduler; import rx.Scheduler.Worker; +import rx.exceptions.Exceptions; import rx.Subscriber; import rx.functions.Action0; @@ -47,7 +48,7 @@ public void call() { try { child.onNext(0L); } catch (Throwable t) { - child.onError(t); + Exceptions.throwOrReport(t, child); return; } child.onCompleted(); diff --git a/src/main/java/rx/internal/operators/OnSubscribeTimerPeriodically.java b/src/main/java/rx/internal/operators/OnSubscribeTimerPeriodically.java index 33811b69e5..bbf14f34a7 100644 --- a/src/main/java/rx/internal/operators/OnSubscribeTimerPeriodically.java +++ b/src/main/java/rx/internal/operators/OnSubscribeTimerPeriodically.java @@ -19,6 +19,7 @@ import rx.Observable.OnSubscribe; import rx.Scheduler; import rx.Scheduler.Worker; +import rx.exceptions.Exceptions; import rx.Subscriber; import rx.functions.Action0; @@ -51,9 +52,9 @@ public void call() { child.onNext(counter++); } catch (Throwable e) { try { - child.onError(e); - } finally { worker.unsubscribe(); + } finally { + Exceptions.throwOrReport(e, child); } } } diff --git a/src/main/java/rx/internal/operators/OnSubscribeToObservableFuture.java b/src/main/java/rx/internal/operators/OnSubscribeToObservableFuture.java index 74adaff15b..72adcf5d50 100644 --- a/src/main/java/rx/internal/operators/OnSubscribeToObservableFuture.java +++ b/src/main/java/rx/internal/operators/OnSubscribeToObservableFuture.java @@ -19,6 +19,7 @@ import java.util.concurrent.TimeUnit; import rx.Observable.OnSubscribe; +import rx.exceptions.Exceptions; import rx.Subscriber; import rx.functions.Action0; import rx.subscriptions.Subscriptions; @@ -83,7 +84,7 @@ public void call() { //refuse to emit onError if already unsubscribed return; } - subscriber.onError(e); + Exceptions.throwOrReport(e, subscriber); } } } diff --git a/src/main/java/rx/internal/operators/OnSubscribeUsing.java b/src/main/java/rx/internal/operators/OnSubscribeUsing.java index 14d8d46b7b..4355e78221 100644 --- a/src/main/java/rx/internal/operators/OnSubscribeUsing.java +++ b/src/main/java/rx/internal/operators/OnSubscribeUsing.java @@ -20,7 +20,7 @@ import rx.*; import rx.Observable.OnSubscribe; -import rx.exceptions.CompositeException; +import rx.exceptions.*; import rx.functions.*; import rx.observers.Subscribers; @@ -72,6 +72,8 @@ public void call(final Subscriber subscriber) { observable.unsafeSubscribe(Subscribers.wrap(subscriber)); } catch (Throwable e) { Throwable disposeError = disposeEagerlyIfRequested(disposeOnceOnly); + Exceptions.throwIfFatal(e); + Exceptions.throwIfFatal(disposeError); if (disposeError != null) subscriber.onError(new CompositeException(Arrays.asList(e, disposeError))); else @@ -80,7 +82,7 @@ public void call(final Subscriber subscriber) { } } catch (Throwable e) { // then propagate error - subscriber.onError(e); + Exceptions.throwOrReport(e, subscriber); } } diff --git a/src/main/java/rx/internal/operators/OperatorAll.java b/src/main/java/rx/internal/operators/OperatorAll.java index 96f0429d01..00845c7334 100644 --- a/src/main/java/rx/internal/operators/OperatorAll.java +++ b/src/main/java/rx/internal/operators/OperatorAll.java @@ -18,7 +18,6 @@ import rx.Observable.Operator; import rx.Subscriber; import rx.exceptions.Exceptions; -import rx.exceptions.OnErrorThrowable; import rx.functions.Func1; import rx.internal.producers.SingleDelayedProducer; @@ -47,8 +46,7 @@ public void onNext(T t) { try { result = predicate.call(t); } catch (Throwable e) { - Exceptions.throwIfFatal(e); - onError(OnErrorThrowable.addValueAsLastCause(e, t)); + Exceptions.throwOrReport(e, this, t); return; } if (!result && !done) { diff --git a/src/main/java/rx/internal/operators/OperatorAny.java b/src/main/java/rx/internal/operators/OperatorAny.java index 7bd9d3f00b..ac84ec961f 100644 --- a/src/main/java/rx/internal/operators/OperatorAny.java +++ b/src/main/java/rx/internal/operators/OperatorAny.java @@ -16,11 +16,9 @@ package rx.internal.operators; -import rx.Observable; +import rx.*; import rx.Observable.Operator; -import rx.Subscriber; import rx.exceptions.Exceptions; -import rx.exceptions.OnErrorThrowable; import rx.functions.Func1; import rx.internal.producers.SingleDelayedProducer; @@ -51,8 +49,7 @@ public void onNext(T t) { try { result = predicate.call(t); } catch (Throwable e) { - Exceptions.throwIfFatal(e); - onError(OnErrorThrowable.addValueAsLastCause(e, t)); + Exceptions.throwOrReport(e, this, t); return; } if (result && !done) { diff --git a/src/main/java/rx/internal/operators/OperatorBufferWithSingleObservable.java b/src/main/java/rx/internal/operators/OperatorBufferWithSingleObservable.java index 204fc365f1..187bc0494a 100644 --- a/src/main/java/rx/internal/operators/OperatorBufferWithSingleObservable.java +++ b/src/main/java/rx/internal/operators/OperatorBufferWithSingleObservable.java @@ -20,6 +20,7 @@ import rx.Observable; import rx.Observable.Operator; +import rx.exceptions.Exceptions; import rx.Observer; import rx.Subscriber; import rx.functions.Func0; @@ -79,7 +80,7 @@ public Subscriber call(final Subscriber> child) { try { closing = bufferClosingSelector.call(); } catch (Throwable t) { - child.onError(t); + Exceptions.throwOrReport(t, child); return Subscribers.empty(); } final BufferingSubscriber bsub = new BufferingSubscriber(new SerializedSubscriber>(child)); @@ -157,7 +158,7 @@ public void onCompleted() { } child.onNext(toEmit); } catch (Throwable t) { - child.onError(t); + Exceptions.throwOrReport(t, child); return; } child.onCompleted(); @@ -183,7 +184,7 @@ void emit() { } done = true; } - child.onError(t); + Exceptions.throwOrReport(t, child); } } } diff --git a/src/main/java/rx/internal/operators/OperatorBufferWithSize.java b/src/main/java/rx/internal/operators/OperatorBufferWithSize.java index 60872b5ba7..d0bfdb1dbb 100644 --- a/src/main/java/rx/internal/operators/OperatorBufferWithSize.java +++ b/src/main/java/rx/internal/operators/OperatorBufferWithSize.java @@ -24,6 +24,7 @@ import rx.Observable.Operator; import rx.Producer; import rx.Subscriber; +import rx.exceptions.Exceptions; /** * This operation takes @@ -118,7 +119,7 @@ public void onCompleted() { try { child.onNext(oldBuffer); } catch (Throwable t) { - onError(t); + Exceptions.throwOrReport(t, this); return; } } @@ -218,7 +219,7 @@ public void onCompleted() { try { child.onNext(chunk); } catch (Throwable t) { - onError(t); + Exceptions.throwOrReport(t, this); return; } } diff --git a/src/main/java/rx/internal/operators/OperatorBufferWithStartEndObservable.java b/src/main/java/rx/internal/operators/OperatorBufferWithStartEndObservable.java index 8e8cb4eeef..328f401a2e 100644 --- a/src/main/java/rx/internal/operators/OperatorBufferWithStartEndObservable.java +++ b/src/main/java/rx/internal/operators/OperatorBufferWithStartEndObservable.java @@ -21,6 +21,7 @@ import java.util.List; import rx.Observable; import rx.Observable.Operator; +import rx.exceptions.Exceptions; import rx.Observer; import rx.Subscriber; import rx.functions.Func1; @@ -145,7 +146,7 @@ public void onCompleted() { child.onNext(chunk); } } catch (Throwable t) { - child.onError(t); + Exceptions.throwOrReport(t, child); return; } child.onCompleted(); @@ -163,7 +164,7 @@ void startBuffer(TOpening v) { try { cobs = bufferClosing.call(v); } catch (Throwable t) { - onError(t); + Exceptions.throwOrReport(t, this); return; } Subscriber closeSubscriber = new Subscriber() { diff --git a/src/main/java/rx/internal/operators/OperatorBufferWithTime.java b/src/main/java/rx/internal/operators/OperatorBufferWithTime.java index 3b2dd63704..bbb723d2b3 100644 --- a/src/main/java/rx/internal/operators/OperatorBufferWithTime.java +++ b/src/main/java/rx/internal/operators/OperatorBufferWithTime.java @@ -25,6 +25,7 @@ import rx.Scheduler; import rx.Scheduler.Worker; import rx.Subscriber; +import rx.exceptions.Exceptions; import rx.functions.Action0; import rx.observers.SerializedSubscriber; @@ -159,7 +160,7 @@ public void onCompleted() { child.onNext(chunk); } } catch (Throwable t) { - child.onError(t); + Exceptions.throwOrReport(t, child); return; } child.onCompleted(); @@ -208,7 +209,7 @@ void emitChunk(List chunkToEmit) { try { child.onNext(chunkToEmit); } catch (Throwable t) { - onError(t); + Exceptions.throwOrReport(t, this); } } } @@ -273,7 +274,7 @@ public void onCompleted() { } child.onNext(toEmit); } catch (Throwable t) { - child.onError(t); + Exceptions.throwOrReport(t, child); return; } child.onCompleted(); @@ -299,7 +300,7 @@ void emit() { try { child.onNext(toEmit); } catch (Throwable t) { - onError(t); + Exceptions.throwOrReport(t, this); } } } diff --git a/src/main/java/rx/internal/operators/OperatorCast.java b/src/main/java/rx/internal/operators/OperatorCast.java index 92dd1792e5..248fcb1970 100644 --- a/src/main/java/rx/internal/operators/OperatorCast.java +++ b/src/main/java/rx/internal/operators/OperatorCast.java @@ -16,8 +16,8 @@ package rx.internal.operators; import rx.Observable.Operator; +import rx.exceptions.*; import rx.Subscriber; -import rx.exceptions.OnErrorThrowable; /** * Converts the elements of an observable sequence to the specified type. @@ -49,7 +49,7 @@ public void onNext(T t) { try { o.onNext(castClass.cast(t)); } catch (Throwable e) { - onError(OnErrorThrowable.addValueAsLastCause(e, t)); + Exceptions.throwOrReport(e, this, t); } } }; diff --git a/src/main/java/rx/internal/operators/OperatorDebounceWithSelector.java b/src/main/java/rx/internal/operators/OperatorDebounceWithSelector.java index c7ae83ff63..6be5ff2210 100644 --- a/src/main/java/rx/internal/operators/OperatorDebounceWithSelector.java +++ b/src/main/java/rx/internal/operators/OperatorDebounceWithSelector.java @@ -17,6 +17,7 @@ import rx.Observable; import rx.Observable.Operator; +import rx.exceptions.Exceptions; import rx.Subscriber; import rx.functions.Func1; import rx.internal.operators.OperatorDebounceWithTime.DebounceState; @@ -59,7 +60,7 @@ public void onNext(T t) { try { debouncer = selector.call(t); } catch (Throwable e) { - onError(e); + Exceptions.throwOrReport(e, this); return; } diff --git a/src/main/java/rx/internal/operators/OperatorDebounceWithTime.java b/src/main/java/rx/internal/operators/OperatorDebounceWithTime.java index 45d3f14cd9..df7c451287 100644 --- a/src/main/java/rx/internal/operators/OperatorDebounceWithTime.java +++ b/src/main/java/rx/internal/operators/OperatorDebounceWithTime.java @@ -19,6 +19,7 @@ import rx.Observable.Operator; import rx.Scheduler; import rx.Scheduler.Worker; +import rx.exceptions.Exceptions; import rx.Subscriber; import rx.functions.Action0; import rx.observers.SerializedSubscriber; @@ -130,7 +131,7 @@ public void emit(int index, Subscriber onNextAndComplete, Subscriber onErr try { onNextAndComplete.onNext(localValue); } catch (Throwable e) { - onError.onError(e); + Exceptions.throwOrReport(e, onError, localValue); return; } @@ -166,7 +167,7 @@ public void emitAndComplete(Subscriber onNextAndComplete, Subscriber onErr try { onNextAndComplete.onNext(localValue); } catch (Throwable e) { - onError.onError(e); + Exceptions.throwOrReport(e, onError, localValue); return; } } diff --git a/src/main/java/rx/internal/operators/OperatorDelayWithSelector.java b/src/main/java/rx/internal/operators/OperatorDelayWithSelector.java index 16744563d7..1c4447c1d2 100644 --- a/src/main/java/rx/internal/operators/OperatorDelayWithSelector.java +++ b/src/main/java/rx/internal/operators/OperatorDelayWithSelector.java @@ -17,6 +17,7 @@ import rx.Observable; import rx.Observable.Operator; +import rx.exceptions.Exceptions; import rx.Subscriber; import rx.functions.Func1; import rx.observers.SerializedSubscriber; @@ -71,7 +72,7 @@ public T call(V v) { })); } catch (Throwable e) { - onError(e); + Exceptions.throwOrReport(e, this); } } diff --git a/src/main/java/rx/internal/operators/OperatorDoOnEach.java b/src/main/java/rx/internal/operators/OperatorDoOnEach.java index 27c3309a1f..4b3e8d54cf 100644 --- a/src/main/java/rx/internal/operators/OperatorDoOnEach.java +++ b/src/main/java/rx/internal/operators/OperatorDoOnEach.java @@ -15,11 +15,9 @@ */ package rx.internal.operators; +import rx.*; import rx.Observable.Operator; -import rx.Observer; -import rx.Subscriber; import rx.exceptions.Exceptions; -import rx.exceptions.OnErrorThrowable; /** * Converts the elements of an observable sequence to the specified type. @@ -45,7 +43,7 @@ public void onCompleted() { try { doOnEachObserver.onCompleted(); } catch (Throwable e) { - onError(e); + Exceptions.throwOrReport(e, this); return; } // Set `done` here so that the error in `doOnEachObserver.onCompleted()` can be noticed by observer @@ -64,7 +62,7 @@ public void onError(Throwable e) { try { doOnEachObserver.onError(e); } catch (Throwable e2) { - observer.onError(e2); + Exceptions.throwOrReport(e2, observer); return; } observer.onError(e); @@ -78,7 +76,7 @@ public void onNext(T value) { try { doOnEachObserver.onNext(value); } catch (Throwable e) { - onError(OnErrorThrowable.addValueAsLastCause(e, value)); + Exceptions.throwOrReport(e, this, value); return; } observer.onNext(value); diff --git a/src/main/java/rx/internal/operators/OperatorFilter.java b/src/main/java/rx/internal/operators/OperatorFilter.java index 276d5f9765..2dbd827a94 100644 --- a/src/main/java/rx/internal/operators/OperatorFilter.java +++ b/src/main/java/rx/internal/operators/OperatorFilter.java @@ -17,7 +17,7 @@ import rx.Observable.Operator; import rx.Subscriber; -import rx.exceptions.OnErrorThrowable; +import rx.exceptions.*; import rx.functions.Func1; /** @@ -57,7 +57,7 @@ public void onNext(T t) { request(1); } } catch (Throwable e) { - child.onError(OnErrorThrowable.addValueAsLastCause(e, t)); + Exceptions.throwOrReport(e, child, t); } } diff --git a/src/main/java/rx/internal/operators/OperatorGroupBy.java b/src/main/java/rx/internal/operators/OperatorGroupBy.java index 3d8f45067c..ffced4c923 100644 --- a/src/main/java/rx/internal/operators/OperatorGroupBy.java +++ b/src/main/java/rx/internal/operators/OperatorGroupBy.java @@ -26,10 +26,10 @@ import rx.Observable; import rx.Observable.OnSubscribe; import rx.Observable.Operator; +import rx.exceptions.*; import rx.Observer; import rx.Producer; import rx.Subscriber; -import rx.exceptions.OnErrorThrowable; import rx.functions.Action0; import rx.functions.Func1; import rx.observables.GroupedObservable; @@ -226,7 +226,7 @@ public void onNext(T t) { emitItem(group, nl.next(t)); } } catch (Throwable e) { - onError(OnErrorThrowable.addValueAsLastCause(e, t)); + Exceptions.throwOrReport(e, this, t); } } @@ -287,7 +287,7 @@ public void onNext(T t) { try { o.onNext(elementSelector.call(t)); } catch (Throwable e) { - onError(OnErrorThrowable.addValueAsLastCause(e, t)); + Exceptions.throwOrReport(e, this, t); } } }); diff --git a/src/main/java/rx/internal/operators/OperatorMap.java b/src/main/java/rx/internal/operators/OperatorMap.java index 1f82a21764..5816887479 100644 --- a/src/main/java/rx/internal/operators/OperatorMap.java +++ b/src/main/java/rx/internal/operators/OperatorMap.java @@ -18,7 +18,6 @@ import rx.Observable.Operator; import rx.Subscriber; import rx.exceptions.Exceptions; -import rx.exceptions.OnErrorThrowable; import rx.functions.Func1; /** @@ -54,8 +53,7 @@ public void onNext(T t) { try { o.onNext(transformer.call(t)); } catch (Throwable e) { - Exceptions.throwIfFatal(e); - onError(OnErrorThrowable.addValueAsLastCause(e, t)); + Exceptions.throwOrReport(e, this, t); } } diff --git a/src/main/java/rx/internal/operators/OperatorMapNotification.java b/src/main/java/rx/internal/operators/OperatorMapNotification.java index bb92f2c077..a0c0994032 100644 --- a/src/main/java/rx/internal/operators/OperatorMapNotification.java +++ b/src/main/java/rx/internal/operators/OperatorMapNotification.java @@ -79,7 +79,7 @@ public void onCompleted() { try { emitter.offerAndComplete(onCompleted.call()); } catch (Throwable e) { - o.onError(e); + Exceptions.throwOrReport(e, o); } } @@ -88,7 +88,7 @@ public void onError(Throwable e) { try { emitter.offerAndComplete(onError.call(e)); } catch (Throwable e2) { - o.onError(e); + Exceptions.throwOrReport(e2, o); } } @@ -97,7 +97,7 @@ public void onNext(T t) { try { emitter.offer(onNext.call(t)); } catch (Throwable e) { - o.onError(OnErrorThrowable.addValueAsLastCause(e, t)); + Exceptions.throwOrReport(e, o, t); } } } diff --git a/src/main/java/rx/internal/operators/OperatorMapPair.java b/src/main/java/rx/internal/operators/OperatorMapPair.java index af95ce1426..29848d2f78 100644 --- a/src/main/java/rx/internal/operators/OperatorMapPair.java +++ b/src/main/java/rx/internal/operators/OperatorMapPair.java @@ -17,8 +17,8 @@ import rx.Observable; import rx.Observable.Operator; +import rx.exceptions.*; import rx.Subscriber; -import rx.exceptions.OnErrorThrowable; import rx.functions.Func1; import rx.functions.Func2; @@ -85,7 +85,7 @@ public R call(U inner) { } })); } catch (Throwable e) { - o.onError(OnErrorThrowable.addValueAsLastCause(e, outer)); + Exceptions.throwOrReport(e, o, outer); } } diff --git a/src/main/java/rx/internal/operators/OperatorOnErrorResumeNextViaFunction.java b/src/main/java/rx/internal/operators/OperatorOnErrorResumeNextViaFunction.java index 70380a1a2b..5141a0974d 100644 --- a/src/main/java/rx/internal/operators/OperatorOnErrorResumeNextViaFunction.java +++ b/src/main/java/rx/internal/operators/OperatorOnErrorResumeNextViaFunction.java @@ -99,7 +99,7 @@ public void setProducer(Producer producer) { Observable resume = resumeFunction.call(e); resume.unsafeSubscribe(next); } catch (Throwable e2) { - child.onError(e2); + Exceptions.throwOrReport(e2, child); } } diff --git a/src/main/java/rx/internal/operators/OperatorOnErrorReturn.java b/src/main/java/rx/internal/operators/OperatorOnErrorReturn.java index 8702093e6c..3830f591fd 100644 --- a/src/main/java/rx/internal/operators/OperatorOnErrorReturn.java +++ b/src/main/java/rx/internal/operators/OperatorOnErrorReturn.java @@ -78,6 +78,7 @@ public void onError(Throwable e) { T result = resultFunction.call(e); child.onNext(result); } catch (Throwable x) { + Exceptions.throwIfFatal(x); child.onError(new CompositeException(Arrays.asList(e, x))); return; } diff --git a/src/main/java/rx/internal/operators/OperatorPublish.java b/src/main/java/rx/internal/operators/OperatorPublish.java index 492cd8f261..65cf83dd25 100644 --- a/src/main/java/rx/internal/operators/OperatorPublish.java +++ b/src/main/java/rx/internal/operators/OperatorPublish.java @@ -19,7 +19,7 @@ import java.util.concurrent.atomic.*; import rx.*; -import rx.exceptions.MissingBackpressureException; +import rx.exceptions.*; import rx.functions.*; import rx.internal.util.*; import rx.internal.util.unsafe.*; @@ -561,7 +561,7 @@ void dispatch() { } catch (Throwable t) { // we bounce back exceptions and kick out the child subscriber ip.unsubscribe(); - ip.child.onError(t); + Exceptions.throwOrReport(t, ip.child, value); continue; } // indicate this child has received 1 element diff --git a/src/main/java/rx/internal/operators/OperatorReplay.java b/src/main/java/rx/internal/operators/OperatorReplay.java index b7b52aded3..6b42f1fb51 100644 --- a/src/main/java/rx/internal/operators/OperatorReplay.java +++ b/src/main/java/rx/internal/operators/OperatorReplay.java @@ -63,8 +63,7 @@ public void call(final Subscriber child) { co = connectableFactory.call(); observable = selector.call(co); } catch (Throwable e) { - Exceptions.throwIfFatal(e); - child.onError(e); + Exceptions.throwOrReport(e, child); return; } diff --git a/src/main/java/rx/internal/operators/OperatorSampleWithTime.java b/src/main/java/rx/internal/operators/OperatorSampleWithTime.java index 7138d760d4..f3130cbb97 100644 --- a/src/main/java/rx/internal/operators/OperatorSampleWithTime.java +++ b/src/main/java/rx/internal/operators/OperatorSampleWithTime.java @@ -20,6 +20,7 @@ import rx.Observable.Operator; import rx.Scheduler; import rx.Scheduler.Worker; +import rx.exceptions.Exceptions; import rx.Subscriber; import rx.functions.Action0; import rx.observers.SerializedSubscriber; @@ -103,7 +104,7 @@ public void call() { T v = (T)localValue; subscriber.onNext(v); } catch (Throwable e) { - onError(e); + Exceptions.throwOrReport(e, this); } } } diff --git a/src/main/java/rx/internal/operators/OperatorSkip.java b/src/main/java/rx/internal/operators/OperatorSkip.java index 878898aaba..505c1491e7 100644 --- a/src/main/java/rx/internal/operators/OperatorSkip.java +++ b/src/main/java/rx/internal/operators/OperatorSkip.java @@ -15,11 +15,7 @@ */ package rx.internal.operators; -import java.util.concurrent.atomic.AtomicBoolean; - -import rx.Observable; -import rx.Producer; -import rx.Subscriber; +import rx.*; /** * Returns an Observable that skips the first num items emitted by the source diff --git a/src/main/java/rx/internal/operators/OperatorTakeLastOne.java b/src/main/java/rx/internal/operators/OperatorTakeLastOne.java index a9bb7b5d33..a7998a1667 100644 --- a/src/main/java/rx/internal/operators/OperatorTakeLastOne.java +++ b/src/main/java/rx/internal/operators/OperatorTakeLastOne.java @@ -3,6 +3,7 @@ import java.util.concurrent.atomic.AtomicInteger; import rx.Observable.Operator; +import rx.exceptions.Exceptions; import rx.Producer; import rx.Subscriber; @@ -150,7 +151,7 @@ private void emit() { try { child.onNext(t); } catch (Throwable e) { - child.onError(e); + Exceptions.throwOrReport(e, child); return; } } diff --git a/src/main/java/rx/internal/operators/OperatorTakeUntilPredicate.java b/src/main/java/rx/internal/operators/OperatorTakeUntilPredicate.java index 668f049a99..c33fab0b47 100644 --- a/src/main/java/rx/internal/operators/OperatorTakeUntilPredicate.java +++ b/src/main/java/rx/internal/operators/OperatorTakeUntilPredicate.java @@ -15,11 +15,10 @@ */ package rx.internal.operators; -import rx.Observable.Operator; import rx.*; +import rx.Observable.Operator; import rx.annotations.Experimental; import rx.exceptions.Exceptions; -import rx.exceptions.OnErrorThrowable; import rx.functions.Func1; /** @@ -47,8 +46,7 @@ public void onNext(T t) { stop = stopPredicate.call(t); } catch (Throwable e) { done = true; - Exceptions.throwIfFatal(e); - child.onError(OnErrorThrowable.addValueAsLastCause(e, t)); + Exceptions.throwOrReport(e, child, t); unsubscribe(); return; } diff --git a/src/main/java/rx/internal/operators/OperatorTakeWhile.java b/src/main/java/rx/internal/operators/OperatorTakeWhile.java index 7d7a219270..0c34df7b6f 100644 --- a/src/main/java/rx/internal/operators/OperatorTakeWhile.java +++ b/src/main/java/rx/internal/operators/OperatorTakeWhile.java @@ -18,11 +18,9 @@ import rx.Observable.Operator; import rx.Subscriber; import rx.exceptions.Exceptions; -import rx.exceptions.OnErrorThrowable; -import rx.functions.Func1; -import rx.functions.Func2; +import rx.functions.*; -/** +/**O * Returns an Observable that emits items emitted by the source Observable as long as a specified * condition is true. *

@@ -60,8 +58,7 @@ public void onNext(T t) { isSelected = predicate.call(t, counter++); } catch (Throwable e) { done = true; - Exceptions.throwIfFatal(e); - subscriber.onError(OnErrorThrowable.addValueAsLastCause(e, t)); + Exceptions.throwOrReport(e, subscriber, t); unsubscribe(); return; } diff --git a/src/main/java/rx/internal/operators/OperatorTimeoutWithSelector.java b/src/main/java/rx/internal/operators/OperatorTimeoutWithSelector.java index ce201c3c26..eff265e4e5 100644 --- a/src/main/java/rx/internal/operators/OperatorTimeoutWithSelector.java +++ b/src/main/java/rx/internal/operators/OperatorTimeoutWithSelector.java @@ -49,8 +49,7 @@ public Subscription call( try { o = firstTimeoutSelector.call(); } catch (Throwable t) { - Exceptions.throwIfFatal(t); - timeoutSubscriber.onError(t); + Exceptions.throwOrReport(t, timeoutSubscriber); return Subscriptions.unsubscribed(); } return o.unsafeSubscribe(new Subscriber() { @@ -85,8 +84,7 @@ public Subscription call( try { o = timeoutSelector.call(value); } catch (Throwable t) { - Exceptions.throwIfFatal(t); - timeoutSubscriber.onError(t); + Exceptions.throwOrReport(t, timeoutSubscriber); return Subscriptions.unsubscribed(); } return o.unsafeSubscribe(new Subscriber() { diff --git a/src/main/java/rx/internal/operators/OperatorToObservableList.java b/src/main/java/rx/internal/operators/OperatorToObservableList.java index e77826acc6..d2e9d717f6 100644 --- a/src/main/java/rx/internal/operators/OperatorToObservableList.java +++ b/src/main/java/rx/internal/operators/OperatorToObservableList.java @@ -18,6 +18,7 @@ import java.util.*; import rx.Observable.Operator; +import rx.exceptions.Exceptions; import rx.*; import rx.internal.producers.SingleDelayedProducer; @@ -85,7 +86,7 @@ public void onCompleted() { */ result = new ArrayList(list); } catch (Throwable t) { - onError(t); + Exceptions.throwOrReport(t, this); return; } list = null; diff --git a/src/main/java/rx/internal/operators/OperatorToObservableSortedList.java b/src/main/java/rx/internal/operators/OperatorToObservableSortedList.java index a3e9c54839..19246cbe7c 100644 --- a/src/main/java/rx/internal/operators/OperatorToObservableSortedList.java +++ b/src/main/java/rx/internal/operators/OperatorToObservableSortedList.java @@ -18,6 +18,7 @@ import java.util.*; import rx.Observable.Operator; +import rx.exceptions.Exceptions; import rx.*; import rx.functions.Func2; import rx.internal.producers.SingleDelayedProducer; @@ -75,7 +76,7 @@ public void onCompleted() { // sort the list before delivery Collections.sort(a, sortFunction); } catch (Throwable e) { - onError(e); + Exceptions.throwOrReport(e, this); return; } producer.setValue(a); diff --git a/src/main/java/rx/internal/operators/OperatorWithLatestFrom.java b/src/main/java/rx/internal/operators/OperatorWithLatestFrom.java index 4bf610b6b1..95a4c30561 100644 --- a/src/main/java/rx/internal/operators/OperatorWithLatestFrom.java +++ b/src/main/java/rx/internal/operators/OperatorWithLatestFrom.java @@ -19,6 +19,7 @@ import rx.*; import rx.Observable.Operator; +import rx.exceptions.Exceptions; import rx.functions.Func2; import rx.observers.SerializedSubscriber; @@ -58,7 +59,7 @@ public void onNext(T t) { s.onNext(result); } catch (Throwable e) { - onError(e); + Exceptions.throwOrReport(e, this); return; } } diff --git a/src/main/java/rx/internal/operators/OperatorZip.java b/src/main/java/rx/internal/operators/OperatorZip.java index 623731755a..d4f0560718 100644 --- a/src/main/java/rx/internal/operators/OperatorZip.java +++ b/src/main/java/rx/internal/operators/OperatorZip.java @@ -20,11 +20,10 @@ import rx.Observable; import rx.Observable.Operator; +import rx.exceptions.*; import rx.Observer; import rx.Producer; import rx.Subscriber; -import rx.exceptions.MissingBackpressureException; -import rx.exceptions.OnErrorThrowable; import rx.functions.Func2; import rx.functions.Func3; import rx.functions.Func4; @@ -265,7 +264,7 @@ void tick() { requested.decrementAndGet(); emitted++; } catch (Throwable e) { - child.onError(OnErrorThrowable.addValueAsLastCause(e, vs)); + Exceptions.throwOrReport(e, child, vs); return; } // now remove them diff --git a/src/main/java/rx/internal/operators/OperatorZipIterable.java b/src/main/java/rx/internal/operators/OperatorZipIterable.java index e73e093082..f913854d1d 100644 --- a/src/main/java/rx/internal/operators/OperatorZipIterable.java +++ b/src/main/java/rx/internal/operators/OperatorZipIterable.java @@ -18,6 +18,7 @@ import java.util.Iterator; import rx.Observable.Operator; +import rx.exceptions.Exceptions; import rx.Subscriber; import rx.functions.Func2; import rx.observers.Subscribers; @@ -41,7 +42,8 @@ public Subscriber call(final Subscriber subscriber) { return Subscribers.empty(); } } catch (Throwable e) { - subscriber.onError(e); + Exceptions.throwOrReport(e, subscriber); + return Subscribers.empty(); } return new Subscriber(subscriber) { boolean once; @@ -67,7 +69,7 @@ public void onNext(T1 t) { onCompleted(); } } catch (Throwable e) { - onError(e); + Exceptions.throwOrReport(e, this); } } diff --git a/src/main/java/rx/internal/operators/TakeLastQueueProducer.java b/src/main/java/rx/internal/operators/TakeLastQueueProducer.java index 633d28ca66..7fc5ce9235 100644 --- a/src/main/java/rx/internal/operators/TakeLastQueueProducer.java +++ b/src/main/java/rx/internal/operators/TakeLastQueueProducer.java @@ -18,6 +18,7 @@ import rx.Producer; import rx.Subscriber; +import rx.exceptions.Exceptions; import java.util.Deque; import java.util.concurrent.atomic.AtomicLongFieldUpdater; @@ -75,7 +76,7 @@ void emit(long previousRequested) { notification.accept(subscriber, value); } } catch (Throwable e) { - subscriber.onError(e); + Exceptions.throwOrReport(e, subscriber); } finally { deque.clear(); } diff --git a/src/main/java/rx/internal/producers/ProducerObserverArbiter.java b/src/main/java/rx/internal/producers/ProducerObserverArbiter.java index ff059590b5..7600815094 100644 --- a/src/main/java/rx/internal/producers/ProducerObserverArbiter.java +++ b/src/main/java/rx/internal/producers/ProducerObserverArbiter.java @@ -233,9 +233,7 @@ void emitLoop() { try { c.onNext(v); } catch (Throwable ex) { - Exceptions.throwIfFatal(ex); - Throwable ex1 = OnErrorThrowable.addValueAsLastCause(ex, v); - c.onError(ex1); + Exceptions.throwOrReport(ex, c, v); return; } } diff --git a/src/main/java/rx/internal/producers/QueuedProducer.java b/src/main/java/rx/internal/producers/QueuedProducer.java index 8dbf4f361e..51747dd9b9 100644 --- a/src/main/java/rx/internal/producers/QueuedProducer.java +++ b/src/main/java/rx/internal/producers/QueuedProducer.java @@ -169,9 +169,7 @@ private void drain() { c.onNext(t); } } catch (Throwable ex) { - Exceptions.throwIfFatal(ex); - Throwable ex1 = OnErrorThrowable.addValueAsLastCause(ex, v != NULL_SENTINEL ? v : null); - c.onError(ex1); + Exceptions.throwOrReport(ex, c, v != NULL_SENTINEL ? v : null); return; } r--; diff --git a/src/main/java/rx/internal/producers/QueuedValueProducer.java b/src/main/java/rx/internal/producers/QueuedValueProducer.java index df61a05041..d165a412b7 100644 --- a/src/main/java/rx/internal/producers/QueuedValueProducer.java +++ b/src/main/java/rx/internal/producers/QueuedValueProducer.java @@ -117,9 +117,7 @@ private void drain() { c.onNext(t); } } catch (Throwable ex) { - Exceptions.throwIfFatal(ex); - Throwable ex1 = OnErrorThrowable.addValueAsLastCause(ex, v != NULL_SENTINEL ? v : null); - c.onError(ex1); + Exceptions.throwOrReport(ex, c, v != NULL_SENTINEL ? v : null); return; } if (c.isUnsubscribed()) { diff --git a/src/main/java/rx/internal/producers/SingleDelayedProducer.java b/src/main/java/rx/internal/producers/SingleDelayedProducer.java index 5da11dd80f..12403fe21b 100644 --- a/src/main/java/rx/internal/producers/SingleDelayedProducer.java +++ b/src/main/java/rx/internal/producers/SingleDelayedProducer.java @@ -101,9 +101,7 @@ private static void emit(Subscriber c, T v) { try { c.onNext(v); } catch (Throwable e) { - Exceptions.throwIfFatal(e); - Throwable e1 = OnErrorThrowable.addValueAsLastCause(e, v); - c.onError(e1); + Exceptions.throwOrReport(e, c, v); return; } if (c.isUnsubscribed()) { diff --git a/src/main/java/rx/internal/producers/SingleProducer.java b/src/main/java/rx/internal/producers/SingleProducer.java index 8e8e17dcb4..337d815d91 100644 --- a/src/main/java/rx/internal/producers/SingleProducer.java +++ b/src/main/java/rx/internal/producers/SingleProducer.java @@ -64,8 +64,7 @@ public void request(long n) { try { c.onNext(v); } catch (Throwable e) { - Exceptions.throwIfFatal(e); - c.onError(OnErrorThrowable.addValueAsLastCause(e, v)); + Exceptions.throwOrReport(e, c, v); return; } // eagerly check for unsubscription diff --git a/src/test/java/rx/internal/operators/OperatorFilterTest.java b/src/test/java/rx/internal/operators/OperatorFilterTest.java index aaa9484be0..f1f086666c 100644 --- a/src/test/java/rx/internal/operators/OperatorFilterTest.java +++ b/src/test/java/rx/internal/operators/OperatorFilterTest.java @@ -16,18 +16,16 @@ package rx.internal.operators; import static org.mockito.Matchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.*; import java.util.concurrent.CountDownLatch; -import org.junit.Test; +import org.junit.*; import org.mockito.Mockito; -import rx.Observable; -import rx.Observer; -import rx.functions.Func1; +import rx.*; +import rx.exceptions.*; +import rx.functions.*; import rx.internal.util.RxRingBuffer; import rx.observers.TestSubscriber; @@ -144,4 +142,29 @@ public void onNext(Integer t) { // this will wait forever unless OperatorTake handles the request(n) on filtered items latch.await(); } + + @Test + public void testFatalError() { + try { + Observable.just(1) + .filter(new Func1() { + @Override + public Boolean call(Integer t) { + return true; + } + }) + .first() + .subscribe(new Action1() { + @Override + public void call(Integer t) { + throw new TestException(); + } + }); + Assert.fail("No exception was thrown"); + } catch (OnErrorNotImplementedException ex) { + if (!(ex.getCause() instanceof TestException)) { + Assert.fail("Failed to report the original exception, instead: " + ex.getCause()); + } + } + } } From 3dcfe18247ad65be81567512f442d4a7be70d269 Mon Sep 17 00:00:00 2001 From: Aaron Tull Date: Tue, 28 Jul 2015 17:55:34 -0700 Subject: [PATCH 49/81] Implementing the SyncOnSubscribe --- .../java/rx/observables/SyncOnSubscribe.java | 465 +++++++++ .../rx/jmh/InputWithIncrementingInteger.java | 22 +- .../observables/BlockingObservablePerf.java | 32 +- src/perf/java/rx/observables/MultiInput.java | 36 + src/perf/java/rx/observables/SingleInput.java | 36 + .../rx/observables/SyncOnSubscribePerf.java | 118 +++ .../rx/observables/SyncOnSubscribeTest.java | 982 ++++++++++++++++++ 7 files changed, 1649 insertions(+), 42 deletions(-) create mode 100644 src/main/java/rx/observables/SyncOnSubscribe.java create mode 100644 src/perf/java/rx/observables/MultiInput.java create mode 100644 src/perf/java/rx/observables/SingleInput.java create mode 100644 src/perf/java/rx/observables/SyncOnSubscribePerf.java create mode 100644 src/test/java/rx/observables/SyncOnSubscribeTest.java diff --git a/src/main/java/rx/observables/SyncOnSubscribe.java b/src/main/java/rx/observables/SyncOnSubscribe.java new file mode 100644 index 0000000000..47a6c34024 --- /dev/null +++ b/src/main/java/rx/observables/SyncOnSubscribe.java @@ -0,0 +1,465 @@ +/** + * Copyright 2014 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.observables; + +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import java.util.concurrent.atomic.AtomicLong; + +import rx.Observable.OnSubscribe; +import rx.Observer; +import rx.Producer; +import rx.Subscriber; +import rx.Subscription; +import rx.annotations.Experimental; +import rx.functions.Action0; +import rx.functions.Action1; +import rx.functions.Action2; +import rx.functions.Func0; +import rx.functions.Func2; +import rx.internal.operators.BackpressureUtils; +import rx.plugins.RxJavaPlugins; + +/** + * A utility class to create {@code OnSubscribe} functions that respond correctly to back + * pressure requests from subscribers. This is an improvement over + * {@link rx.Observable#create(OnSubscribe) Observable.create(OnSubscribe)} which does not provide + * any means of managing back pressure requests out-of-the-box. + * + * @param + * the type of the user-define state used in {@link #generateState() generateState(S)} , + * {@link #next(Object, Subscriber) next(S, Subscriber)}, and + * {@link #onUnsubscribe(Object) onUnsubscribe(S)}. + * @param + * the type of {@code Subscribers} that will be compatible with {@code this}. + */ +@Experimental +public abstract class SyncOnSubscribe implements OnSubscribe { + + /* (non-Javadoc) + * @see rx.functions.Action1#call(java.lang.Object) + */ + @Override + public final void call(final Subscriber subscriber) { + S state = generateState(); + SubscriptionProducer p = new SubscriptionProducer(subscriber, this, state); + subscriber.add(p); + subscriber.setProducer(p); + } + + /** + * Executed once when subscribed to by a subscriber (via {@link OnSubscribe#call(Subscriber)}) + * to produce a state value. This value is passed into {@link #next(Object, Observer) next(S + * state, Observer observer)} on the first iteration. Subsequent iterations of {@code next} + * will receive the state returned by the previous invocation of {@code next}. + * + * @return the initial state value + */ + protected abstract S generateState(); + + /** + * Called to produce data to the downstream subscribers. To emit data to a downstream subscriber + * call {@code observer.onNext(t)}. To signal an error condition call + * {@code observer.onError(throwable)} or throw an Exception. To signal the end of a data stream + * call {@code + * observer.onCompleted()}. Implementations of this method must follow the following rules. + * + *

    + *
  • Must not call {@code observer.onNext(t)} more than 1 time per invocation.
  • + *
  • Must not call {@code observer.onNext(t)} concurrently.
  • + *
+ * + * The value returned from an invocation of this method will be passed in as the {@code state} + * argument of the next invocation of this method. + * + * @param state + * the state value (from {@link #generateState()} on the first invocation or the + * previous invocation of this method. + * @param observer + * the observer of data emitted by + * @return the next iteration's state value + */ + protected abstract S next(S state, Observer observer); + + /** + * Clean up behavior that is executed after the downstream subscriber's subscription is + * unsubscribed. This method will be invoked exactly once. + * + * @param state + * the last state value prior from {@link #generateState()} or + * {@link #next(Object, Observer) next(S, Observer<T>)} before unsubscribe. + */ + protected void onUnsubscribe(S state) { + + } + + /** + * Generates a synchronous {@link SyncOnSubscribe} that calls the provided {@code next} function + * to generate data to downstream subscribers. + * + * @param generator + * generates the initial state value (see {@link #generateState()}) + * @param next + * produces data to the downstream subscriber (see {@link #next(Object, Subscriber) + * next(S, Subscriber)}) + * @return an OnSubscribe that emits data in a protocol compatible with back-pressure. + */ + @Experimental + public static OnSubscribe createSingleState(Func0 generator, + final Action2> next) { + Func2, S> nextFunc = new Func2, S>() { + @Override + public S call(S state, Observer subscriber) { + next.call(state, subscriber); + return state; + } + }; + return new SyncOnSubscribeImpl(generator, nextFunc); + } + + /** + * Generates a synchronous {@link SyncOnSubscribe} that calls the provided {@code next} function + * to generate data to downstream subscribers. + * + * This overload creates a SyncOnSubscribe without an explicit clean up step. + * + * @param generator + * generates the initial state value (see {@link #generateState()}) + * @param next + * produces data to the downstream subscriber (see {@link #next(Object, Subscriber) + * next(S, Subscriber)}) + * @param onUnsubscribe + * clean up behavior (see {@link #onUnsubscribe(Object) onUnsubscribe(S)}) + * @return an OnSubscribe that emits data downstream in a protocol compatible with + * back-pressure. + */ + @Experimental + public static OnSubscribe createSingleState(Func0 generator, + final Action2> next, + final Action1 onUnsubscribe) { + Func2, S> nextFunc = new Func2, S>() { + @Override + public S call(S state, Observer subscriber) { + next.call(state, subscriber); + return state; + } + }; + return new SyncOnSubscribeImpl(generator, nextFunc, onUnsubscribe); + } + + /** + * Generates a synchronous {@link SyncOnSubscribe} that calls the provided {@code next} function + * to generate data to downstream subscribers. + * + * @param generator + * generates the initial state value (see {@link #generateState()}) + * @param next + * produces data to the downstream subscriber (see {@link #next(Object, Subscriber) + * next(S, Subscriber)}) + * @param onUnsubscribe + * clean up behavior (see {@link #onUnsubscribe(Object) onUnsubscribe(S)}) + * @return an OnSubscribe that emits data downstream in a protocol compatible with + * back-pressure. + */ + @Experimental + public static OnSubscribe createStateful(Func0 generator, + Func2, ? extends S> next, + Action1 onUnsubscribe) { + return new SyncOnSubscribeImpl(generator, next, onUnsubscribe); + } + + /** + * Generates a synchronous {@link SyncOnSubscribe} that calls the provided {@code next} function + * to generate data to downstream subscribers. + * + * @param generator + * generates the initial state value (see {@link #generateState()}) + * @param next + * produces data to the downstream subscriber (see {@link #next(Object, Subscriber) + * next(S, Subscriber)}) + * @return an OnSubscribe that emits data downstream in a protocol compatible with + * back-pressure. + */ + @Experimental + public static OnSubscribe createStateful(Func0 generator, + Func2, ? extends S> next) { + return new SyncOnSubscribeImpl(generator, next); + } + + /** + * Generates a synchronous {@link SyncOnSubscribe} that calls the provided {@code next} function + * to generate data to downstream subscribers. + * + * This overload creates a "state-less" SyncOnSubscribe which does not have an explicit state + * value. This should be used when the {@code next} function closes over it's state. + * + * @param next + * produces data to the downstream subscriber (see {@link #next(Object, Subscriber) + * next(S, Subscriber)}) + * @return an OnSubscribe that emits data downstream in a protocol compatible with + * back-pressure. + */ + @Experimental + public static OnSubscribe createStateless(final Action1> next) { + Func2, Void> nextFunc = new Func2, Void>() { + @Override + public Void call(Void state, Observer subscriber) { + next.call(subscriber); + return state; + } + }; + return new SyncOnSubscribeImpl(nextFunc); + } + + /** + * Generates a synchronous {@link SyncOnSubscribe} that calls the provided {@code next} function + * to generate data to downstream subscribers. + * + * This overload creates a "state-less" SyncOnSubscribe which does not have an explicit state + * value. This should be used when the {@code next} function closes over it's state. + * + * @param next + * produces data to the downstream subscriber (see {@link #next(Object, Subscriber) + * next(S, Subscriber)}) + * @param onUnsubscribe + * clean up behavior (see {@link #onUnsubscribe(Object) onUnsubscribe(S)}) + * @return an OnSubscribe that emits data downstream in a protocol compatible with + * back-pressure. + */ + @Experimental + public static OnSubscribe createStateless(final Action1> next, + final Action0 onUnsubscribe) { + Func2, Void> nextFunc = new Func2, Void>() { + @Override + public Void call(Void state, Observer subscriber) { + next.call(subscriber); + return null; + } + }; + Action1 wrappedOnUnsubscribe = new Action1(){ + @Override + public void call(Void t) { + onUnsubscribe.call(); + }}; + return new SyncOnSubscribeImpl(nextFunc, wrappedOnUnsubscribe); + } + + /** + * An implementation of SyncOnSubscribe that delegates + * {@link SyncOnSubscribe#next(Object, Subscriber)}, {@link SyncOnSubscribe#generateState()}, + * and {@link SyncOnSubscribe#onUnsubscribe(Object)} to provided functions/closures. + * + * @param + * the type of the user-defined state + * @param + * the type of compatible Subscribers + */ + private static final class SyncOnSubscribeImpl extends SyncOnSubscribe { + private final Func0 generator; + private final Func2, ? extends S> next; + private final Action1 onUnsubscribe; + + private SyncOnSubscribeImpl(Func0 generator, Func2, ? extends S> next, Action1 onUnsubscribe) { + this.generator = generator; + this.next = next; + this.onUnsubscribe = onUnsubscribe; + } + + public SyncOnSubscribeImpl(Func0 generator, Func2, ? extends S> next) { + this(generator, next, null); + } + + public SyncOnSubscribeImpl(Func2, S> next, Action1 onUnsubscribe) { + this(null, next, onUnsubscribe); + } + + public SyncOnSubscribeImpl(Func2, S> nextFunc) { + this(null, nextFunc, null); + } + + @Override + protected S generateState() { + return generator == null ? null : generator.call(); + } + + @Override + protected S next(S state, Observer observer) { + return next.call(state, observer); + } + + @Override + protected void onUnsubscribe(S state) { + if (onUnsubscribe != null) + onUnsubscribe.call(state); + } + } + + /** + * Contains the producer loop that reacts to downstream requests of work. + * + * @param + * the type of compatible Subscribers + */ + private static class SubscriptionProducer + extends AtomicLong implements Producer, Subscription, Observer { + /** */ + private static final long serialVersionUID = -3736864024352728072L; + private final Subscriber actualSubscriber; + private final SyncOnSubscribe parent; + private boolean onNextCalled; + private boolean hasTerminated; + + private S state; + + volatile int isUnsubscribed; + @SuppressWarnings("rawtypes") + static final AtomicIntegerFieldUpdater IS_UNSUBSCRIBED = + AtomicIntegerFieldUpdater.newUpdater(SubscriptionProducer.class, "isUnsubscribed"); + + private SubscriptionProducer(final Subscriber subscriber, SyncOnSubscribe parent, S state) { + this.actualSubscriber = subscriber; + this.parent = parent; + this.state = state; + } + + @Override + public boolean isUnsubscribed() { + return isUnsubscribed != 0; + } + + @Override + public void unsubscribe() { + IS_UNSUBSCRIBED.compareAndSet(this, 0, 1); + if (get() == 0L) + parent.onUnsubscribe(state); + } + + @Override + public void request(long n) { + if (n > 0 && BackpressureUtils.getAndAddRequest(this, n) == 0L) { + if (n == Long.MAX_VALUE) { + fastpath(); + } else { + slowPath(n); + } + } + } + + void fastpath() { + final SyncOnSubscribe p = parent; + Subscriber a = actualSubscriber; + + if (isUnsubscribed()) { + p.onUnsubscribe(state); + return; + } + + for (;;) { + try { + onNextCalled = false; + nextIteration(p); + } catch (Throwable ex) { + handleThrownError(p, a, state, ex); + return; + } + if (hasTerminated || isUnsubscribed()) { + p.onUnsubscribe(state); + return; + } + } + } + + private void handleThrownError(final SyncOnSubscribe p, Subscriber a, S st, Throwable ex) { + if (hasTerminated) { + RxJavaPlugins.getInstance().getErrorHandler().handleError(ex); + } else { + hasTerminated = true; + a.onError(ex); + p.onUnsubscribe(st); + } + } + + void slowPath(long n) { + final SyncOnSubscribe p = parent; + Subscriber a = actualSubscriber; + long numRequested = n; + for (;;) { + if (isUnsubscribed()) { + p.onUnsubscribe(state); + return; + } + long numRemaining = numRequested; + do { + try { + onNextCalled = false; + nextIteration(p); + } catch (Throwable ex) { + handleThrownError(p, a, state, ex); + return; + } + if (hasTerminated || isUnsubscribed()) { + p.onUnsubscribe(state); + return; + } + if (onNextCalled) + numRemaining--; + } while (numRemaining != 0L); + + numRequested = addAndGet(-numRequested); + if (numRequested == 0L) { + break; + } + } + } + + private void nextIteration(final SyncOnSubscribe parent) { + state = parent.next(state, this); + } + + @Override + public void onCompleted() { + if (hasTerminated) { + throw new IllegalStateException("Terminal event already emitted."); + } + hasTerminated = true; + if (!actualSubscriber.isUnsubscribed()) { + actualSubscriber.onCompleted(); + } + } + + @Override + public void onError(Throwable e) { + if (hasTerminated) { + throw new IllegalStateException("Terminal event already emitted."); + } + hasTerminated = true; + if (!actualSubscriber.isUnsubscribed()) { + actualSubscriber.onError(e); + } + } + + @Override + public void onNext(T value) { + if (onNextCalled) { + throw new IllegalStateException("onNext called multiple times!"); + } + onNextCalled = true; + actualSubscriber.onNext(value); + } + } + + +} \ No newline at end of file diff --git a/src/perf/java/rx/jmh/InputWithIncrementingInteger.java b/src/perf/java/rx/jmh/InputWithIncrementingInteger.java index f86bc28117..6760202024 100644 --- a/src/perf/java/rx/jmh/InputWithIncrementingInteger.java +++ b/src/perf/java/rx/jmh/InputWithIncrementingInteger.java @@ -13,6 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package rx.jmh; import java.util.Iterator; @@ -40,46 +41,43 @@ public abstract class InputWithIncrementingInteger { @Setup public void setup(final Blackhole bh) { this.bh = bh; - observable = Observable.range(0, getSize()); + final int size = getSize(); + observable = Observable.range(0, size); firehose = Observable.create(new OnSubscribe() { @Override public void call(Subscriber s) { - for (int i = 0; i < getSize(); i++) { + for (int i = 0; i < size; i++) { s.onNext(i); } s.onCompleted(); } }); - iterable = new Iterable() { - @Override public Iterator iterator() { return new Iterator() { - int i = 0; - + @Override public boolean hasNext() { - return i < getSize(); + return i < size; } - + @Override public Integer next() { + Blackhole.consumeCPU(10); return i++; } - + @Override public void remove() { - + } - }; } - }; observer = new Observer() { diff --git a/src/perf/java/rx/observables/BlockingObservablePerf.java b/src/perf/java/rx/observables/BlockingObservablePerf.java index 4cb18d31c0..7c6b00029e 100644 --- a/src/perf/java/rx/observables/BlockingObservablePerf.java +++ b/src/perf/java/rx/observables/BlockingObservablePerf.java @@ -15,48 +15,20 @@ */ package rx.observables; +import java.util.concurrent.TimeUnit; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Mode; import org.openjdk.jmh.annotations.OutputTimeUnit; -import org.openjdk.jmh.annotations.Param; import org.openjdk.jmh.annotations.Scope; import org.openjdk.jmh.annotations.State; -import rx.jmh.InputWithIncrementingInteger; - -import java.util.concurrent.TimeUnit; @BenchmarkMode(Mode.Throughput) @OutputTimeUnit(TimeUnit.SECONDS) @State(Scope.Thread) public class BlockingObservablePerf { - @State(Scope.Thread) - public static class MultiInput extends InputWithIncrementingInteger { - - @Param({ "1", "1000", "1000000" }) - public int size; - - @Override - public int getSize() { - return size; - } - - } - - @State(Scope.Thread) - public static class SingleInput extends InputWithIncrementingInteger { - - @Param({ "1" }) - public int size; - - @Override - public int getSize() { - return size; - } - - } - @Benchmark public int benchSingle(final SingleInput input) { return input.observable.toBlocking().single(); diff --git a/src/perf/java/rx/observables/MultiInput.java b/src/perf/java/rx/observables/MultiInput.java new file mode 100644 index 0000000000..e607249d07 --- /dev/null +++ b/src/perf/java/rx/observables/MultiInput.java @@ -0,0 +1,36 @@ +/** + * Copyright 2014 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.observables; + +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.State; + +import rx.jmh.InputWithIncrementingInteger; + +@State(Scope.Thread) +public class MultiInput extends InputWithIncrementingInteger { + + @Param({ "1", "1000", "1000000" }) + public int size; + + @Override + public int getSize() { + return size; + } + +} diff --git a/src/perf/java/rx/observables/SingleInput.java b/src/perf/java/rx/observables/SingleInput.java new file mode 100644 index 0000000000..7949efcfa5 --- /dev/null +++ b/src/perf/java/rx/observables/SingleInput.java @@ -0,0 +1,36 @@ +/** + * Copyright 2014 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.observables; + +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.State; + +import rx.jmh.InputWithIncrementingInteger; + +@State(Scope.Thread) +public class SingleInput extends InputWithIncrementingInteger { + + @Param({ "1" }) + public int size; + + @Override + public int getSize() { + return size; + } + +} diff --git a/src/perf/java/rx/observables/SyncOnSubscribePerf.java b/src/perf/java/rx/observables/SyncOnSubscribePerf.java new file mode 100644 index 0000000000..8417bf3a8e --- /dev/null +++ b/src/perf/java/rx/observables/SyncOnSubscribePerf.java @@ -0,0 +1,118 @@ +/** + * Copyright 2014 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.observables; + +import java.util.Iterator; +import java.util.concurrent.TimeUnit; + +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.infra.Blackhole; + +import rx.Observable.OnSubscribe; +import rx.Observer; +import rx.internal.operators.OnSubscribeFromIterable; + +@BenchmarkMode(Mode.Throughput) +@OutputTimeUnit(TimeUnit.SECONDS) +@State(Scope.Thread) +public class SyncOnSubscribePerf { + + public static void main(String[] args) { + SingleInput singleInput = new SingleInput(); + singleInput.size = 1; + singleInput.setup(generated._jmh_tryInit_()); + SyncOnSubscribePerf perf = new SyncOnSubscribePerf(); + perf.benchSyncOnSubscribe(singleInput); + } + private static class generated { + private static Blackhole _jmh_tryInit_() { + return new Blackhole(); + } + } + + private static OnSubscribe createSyncOnSubscribe(final Iterator iterator) { + return new SyncOnSubscribe(){ + + @Override + protected Void generateState() { + return null; + } + + @Override + protected Void next(Void state, Observer observer) { + if (iterator.hasNext()) { + observer.onNext(iterator.next()); + } + else + observer.onCompleted(); + return null; + } + }; + } + +// @Benchmark +// @Group("single") + public void benchSyncOnSubscribe(final SingleInput input) { + createSyncOnSubscribe(input.iterable.iterator()).call(input.newSubscriber()); + } + +// @Benchmark +// @Group("single") + public void benchFromIterable(final SingleInput input) { + new OnSubscribeFromIterable(input.iterable).call(input.newSubscriber()); + } + +// @Benchmark +// @Group("single") + public void benchAbstractOnSubscribe(final SingleInput input) { + final Iterator iterator = input.iterable.iterator(); + createAbstractOnSubscribe(iterator).call(input.newSubscriber()); + } + + private AbstractOnSubscribe createAbstractOnSubscribe(final Iterator iterator) { + return new AbstractOnSubscribe() { + @Override + protected void next(rx.observables.AbstractOnSubscribe.SubscriptionState state) { + if (iterator.hasNext()) + state.onNext(iterator.next()); + else + state.onCompleted(); + }}; + } + + @Benchmark +// @Group("multi") + public void benchSyncOnSubscribe2(final MultiInput input) { + createSyncOnSubscribe(input.iterable.iterator()).call(input.newSubscriber()); + } + +// @Benchmark +// @Group("multi") + public void benchAbstractOnSubscribe2(final MultiInput input) { + createAbstractOnSubscribe(input.iterable.iterator()).call(input.newSubscriber()); + } + + @Benchmark +// @Group("multi") + public void benchFromIterable2(final MultiInput input) { + new OnSubscribeFromIterable(input.iterable).call(input.newSubscriber()); + } +} diff --git a/src/test/java/rx/observables/SyncOnSubscribeTest.java b/src/test/java/rx/observables/SyncOnSubscribeTest.java new file mode 100644 index 0000000000..91421d502a --- /dev/null +++ b/src/test/java/rx/observables/SyncOnSubscribeTest.java @@ -0,0 +1,982 @@ +/** + * Copyright 2014 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.observables; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertFalse; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.isA; +import static org.mockito.Mockito.inOrder; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +import org.junit.Test; +import org.mockito.InOrder; +import org.mockito.Matchers; +import org.mockito.Mockito; + +import rx.Observable; +import rx.Observable.OnSubscribe; +import rx.Observable.Operator; +import rx.Observer; +import rx.Producer; +import rx.Subscriber; +import rx.exceptions.TestException; +import rx.functions.Action0; +import rx.functions.Action1; +import rx.functions.Action2; +import rx.functions.Func0; +import rx.functions.Func2; +import rx.observers.TestSubscriber; +import rx.schedulers.Schedulers; +import rx.schedulers.TestScheduler; + +/** + * Test if SyncOnSubscribe adheres to the usual unsubscription and backpressure contracts. + */ +public class SyncOnSubscribeTest { + @Test + public void testObservableJustEquivalent() { + OnSubscribe os = SyncOnSubscribe.createStateless(new Action1>() { + @Override + public void call(Observer subscriber) { + subscriber.onNext(1); + subscriber.onCompleted(); + }}); + + TestSubscriber ts = new TestSubscriber(); + + Observable.create(os).subscribe(ts); + + ts.assertNoErrors(); + ts.assertTerminalEvent(); + ts.assertReceivedOnNext(Arrays.asList(1)); + } + + @Test + public void testStateAfterTerminal() { + final AtomicInteger finalStateValue = new AtomicInteger(-1); + OnSubscribe os = SyncOnSubscribe.createStateful(new Func0(){ + @Override + public Integer call() { + return 1; + }}, + new Func2, Integer>() { + @Override + public Integer call(Integer state, Observer subscriber) { + subscriber.onNext(state); + subscriber.onCompleted(); + return state + 1; + }}, new Action1() { + @Override + public void call(Integer t) { + finalStateValue.set(t); + }}); + + TestSubscriber ts = new TestSubscriber(); + + Observable.create(os).subscribe(ts); + + ts.assertNoErrors(); + ts.assertTerminalEvent(); + ts.assertValue(1); + assertEquals(2, finalStateValue.get()); + } + + @Test + public void testMultipleOnNextValuesCallsOnError() { + OnSubscribe os = SyncOnSubscribe.createStateless(new Action1>() { + @Override + public void call(Observer subscriber) { + subscriber.onNext(1); + subscriber.onNext(2); + subscriber.onCompleted(); + }}); + + @SuppressWarnings("unchecked") + Observer o = mock(Observer.class); + + Observable.create(os).subscribe(o); + + verify(o, times(1)).onNext(1); + verify(o, never()).onNext(2); + verify(o, never()).onCompleted(); + verify(o, times(1)).onError(any(IllegalStateException.class)); + } + + @Test + public void testMultipleOnCompleted() { + OnSubscribe os = SyncOnSubscribe.createStateless(new Action1>() { + @Override + public void call(Observer subscriber) { + subscriber.onNext(1); + subscriber.onCompleted(); + subscriber.onCompleted(); + }}); + + @SuppressWarnings("unchecked") + Observer o = mock(Observer.class); + + Observable.create(os).subscribe(o); + + verify(o, times(1)).onNext(1); + verify(o, times(1)).onCompleted(); + verify(o, never()).onError(any(Throwable.class)); + } + + @Test + public void testOnNextAfterOnComplete() { + OnSubscribe os = SyncOnSubscribe.createStateless(new Action1>() { + @Override + public void call(Observer subscriber) { + subscriber.onNext(1); + subscriber.onCompleted(); + subscriber.onNext(1); + }}); + + @SuppressWarnings("unchecked") + Observer o = mock(Observer.class); + + Observable.create(os).subscribe(o); + + verify(o, times(1)).onNext(1); + verify(o, times(1)).onCompleted(); + verify(o, never()).onError(any(Throwable.class)); + } + + @SuppressWarnings("serial") + private static class FooException extends RuntimeException { + public FooException(String string) { + super(string); + } + } + + @Test + public void testMultipleOnErrors() { + OnSubscribe os = SyncOnSubscribe.createStateless(new Action1>() { + @Override + public void call(Observer subscriber) { + subscriber.onNext(1); + subscriber.onError(new TestException("Forced failure 1")); + subscriber.onError(new FooException("Should not see this error.")); + }}); + + @SuppressWarnings("unchecked") + Observer o = mock(Observer.class); + + Observable.create(os).subscribe(o); + + verify(o, times(1)).onNext(1); + verify(o, never()).onCompleted(); + verify(o, times(1)).onError(isA(TestException.class)); + verify(o, never()).onError(isA(FooException.class)); + } + + @Test + public void testEmpty() { + OnSubscribe os = SyncOnSubscribe.createStateless(new Action1>() { + @Override + public void call(Observer subscriber) { + subscriber.onCompleted(); + }}); + + @SuppressWarnings("unchecked") + Observer o = mock(Observer.class); + + Observable.create(os).subscribe(o); + + verify(o, never()).onNext(any(Integer.class)); + verify(o, never()).onError(any(Throwable.class)); + verify(o).onCompleted(); + } + + @Test + public void testNever() { + OnSubscribe os = SyncOnSubscribe.createStateless(new Action1>() { + @Override + public void call(Observer subscriber) { + + }}); + + + Observable neverObservable = Observable.create(os).subscribeOn(Schedulers.newThread()); + Observable merged = Observable.amb(neverObservable, Observable.timer(100, TimeUnit.MILLISECONDS).subscribeOn(Schedulers.newThread())); + Iterator values = merged.toBlocking().toIterable().iterator(); + + assertTrue((values.hasNext())); + assertEquals(0l, values.next()); + } + + @Test + public void testThrows() { + OnSubscribe os = SyncOnSubscribe.createStateless(new Action1>() { + @Override + public void call(Observer subscriber) { + throw new TestException("Forced failure"); + }}); + + @SuppressWarnings("unchecked") + Observer o = mock(Observer.class); + + Observable.create(os).subscribe(o); + + verify(o, never()).onNext(any(Integer.class)); + verify(o, never()).onCompleted(); + verify(o).onError(any(TestException.class)); + } + + @Test + public void testThrowAfterCompleteFastPath() { + OnSubscribe os = SyncOnSubscribe.createStateless(new Action1>() { + @Override + public void call(Observer subscriber) { + subscriber.onCompleted(); + throw new TestException("Forced failure"); + }}); + @SuppressWarnings("unchecked") + Observer o = mock(Observer.class); + + Observable.create(os).subscribe(o); + + verify(o, never()).onNext(any(Integer.class)); + verify(o, times(1)).onCompleted(); + verify(o, never()).onError(any(Throwable.class)); + } + + @Test + public void testThrowsSlowPath() { + OnSubscribe os = SyncOnSubscribe.createStateless(new Action1>() { + @Override + public void call(Observer subscriber) { + throw new TestException("Forced failure"); + }}); + + @SuppressWarnings("unchecked") + Observer o = mock(Observer.class); + + TestSubscriber ts = new TestSubscriber(o) { + @Override + public void onStart() { + requestMore(0); // don't start right away + } + }; + + Observable.create(os).subscribe(ts); + + ts.requestMore(1); + + verify(o, never()).onNext(any(Integer.class)); + verify(o, never()).onCompleted(); + verify(o, times(1)).onError(any(TestException.class)); + } + + @Test + public void testError() { + OnSubscribe os = SyncOnSubscribe.createStateless(new Action1>() { + @Override + public void call(Observer subscriber) { + subscriber.onError(new TestException("Forced failure")); + }}); + + @SuppressWarnings("unchecked") + Observer o = mock(Observer.class); + + Observable.create(os).subscribe(o); + + verify(o, never()).onNext(any(Integer.class)); + verify(o).onError(any(TestException.class)); + verify(o, never()).onCompleted(); + } + + @Test + public void testRange() { + final int start = 1; + final int count = 4000; + OnSubscribe os = SyncOnSubscribe.createStateful(new Func0(){ + @Override + public Integer call() { + return start; + }}, + new Func2, Integer>() { + @Override + public Integer call(Integer state, Observer subscriber) { + subscriber.onNext(state); + if (state == count) { + subscriber.onCompleted(); + } + return state + 1; + } + }); + + @SuppressWarnings("unchecked") + Observer o = mock(Observer.class); + InOrder inOrder = inOrder(o); + + Observable.create(os).subscribe(o); + + verify(o, never()).onError(any(TestException.class)); + inOrder.verify(o, times(count)).onNext(any(Integer.class)); + inOrder.verify(o).onCompleted(); + inOrder.verifyNoMoreInteractions(); + } + + @Test + public void testFromIterable() { + int n = 400; + final List source = new ArrayList(); + for (int i = 0; i < n; i++) { + source.add(i); + } + OnSubscribe os = SyncOnSubscribe.createStateful( + new Func0>() { + @Override + public Iterator call() { + return source.iterator(); + }}, + new Func2, Observer, Iterator>() { + @Override + public Iterator call(Iterator it, Observer observer) { + if (it.hasNext()) { + observer.onNext(it.next()); + } + if (!it.hasNext()) { + observer.onCompleted(); + } + return it; + }}); + + + @SuppressWarnings("unchecked") + Observer o = mock(Observer.class); + InOrder inOrder = inOrder(o); + + Observable.create(os).subscribe(o); + + verify(o, never()).onError(any(TestException.class)); + inOrder.verify(o, times(n)).onNext(any()); + inOrder.verify(o).onCompleted(); + inOrder.verifyNoMoreInteractions(); + } + + @Test + public void testInfiniteTake() { + final int start = 0; + final int finalCount = 4000; + OnSubscribe os = SyncOnSubscribe.createStateful( + new Func0() { + @Override + public Integer call() { + return start; + }}, + new Func2, Integer>() { + @Override + public Integer call(Integer state, Observer observer) { + observer.onNext(state); + return state + 1; + }}); + @SuppressWarnings("unchecked") + Observer o = mock(Observer.class); + InOrder inOrder = inOrder(o); + + Observable.create(os).take(finalCount).subscribe(o); + + verify(o, never()).onError(any(Throwable.class)); + inOrder.verify(o, times(finalCount)).onNext(any()); + inOrder.verify(o).onCompleted(); + inOrder.verifyNoMoreInteractions(); + } + + @Test + public void testInfiniteRequestSome() { + final int finalCount = 4000; + final int start = 0; + + @SuppressWarnings("unchecked") + Action1 onUnSubscribe = mock(Action1.class); + + OnSubscribe os = SyncOnSubscribe.createStateful( + new Func0() { + @Override + public Integer call() { + return start; + }}, + new Func2, Integer>() { + @Override + public Integer call(Integer state, Observer observer) { + observer.onNext(state); + return state + 1; + }}, + onUnSubscribe); + + @SuppressWarnings("unchecked") + Observer o = mock(Observer.class); + InOrder inOrder = inOrder(o); + + TestSubscriber ts = new TestSubscriber(o) { + @Override + public void onStart() { + requestMore(0); // don't start right away + } + }; + + Observable.create(os).subscribe(ts); + + ts.requestMore(finalCount); + + verify(o, never()).onError(any(Throwable.class)); + verify(o, never()).onCompleted(); + inOrder.verify(o, times(finalCount)).onNext(any()); + inOrder.verifyNoMoreInteractions(); + // unsubscribe does not take place because subscriber is still in process of requesting + verify(onUnSubscribe, never()).call(any(Integer.class)); + } + + @Test + public void testUnsubscribeDownstream() { + @SuppressWarnings("unchecked") + Action1 onUnSubscribe = mock(Action1.class); + + OnSubscribe os = SyncOnSubscribe.createStateful( + new Func0() { + @Override + public Integer call() { + return null; + }}, + new Func2, Integer>() { + @Override + public Integer call(Integer state, Observer observer) { + observer.onNext(state); + return state; + }}, + onUnSubscribe); + + @SuppressWarnings("unchecked") + Observer o = mock(Observer.class); + + TestSubscriber ts = new TestSubscriber(o); + + Observable.create(os).lift(new Operator(){ + @Override + public Subscriber call(final Subscriber subscriber) { + return new Subscriber(){ + @Override + public void setProducer(Producer p) { + p.request(Long.MAX_VALUE); + } + + @Override + public void onCompleted() { + subscriber.onCompleted(); + } + + @Override + public void onError(Throwable e) { + subscriber.onError(e); + } + + @Override + public void onNext(Object t) { + subscriber.onNext(t); + unsubscribe(); + }}; + }}).take(1).subscribe(ts); + + verify(o, never()).onError(any(Throwable.class)); + verify(onUnSubscribe, times(1)).call(any(Integer.class)); + } + + @Test + public void testConcurrentRequests() throws InterruptedException { + final int count1 = 1000; + final int count2 = 1000; + final int finalCount = count1 + count2; + final int start = 1; + final CountDownLatch l1 = new CountDownLatch(1); + final CountDownLatch l2 = new CountDownLatch(1); + + @SuppressWarnings("unchecked") + Action1 onUnSubscribe = mock(Action1.class); + + OnSubscribe os = SyncOnSubscribe.createStateful( + new Func0() { + @Override + public Integer call() { + return start; + }}, + new Func2, Integer>() { + @Override + public Integer call(Integer state, Observer observer) { + // countdown so the other thread is certain to make a concurrent request + l2.countDown(); + // wait until the 2nd request returns then proceed + try { + if (!l1.await(1, TimeUnit.SECONDS)) + throw new IllegalStateException(); + } catch (InterruptedException e) {} + observer.onNext(state); + if (state == finalCount) + observer.onCompleted(); + return state + 1; + }}, + onUnSubscribe); + + @SuppressWarnings("unchecked") + Observer o = mock(Observer.class); + InOrder inOrder = inOrder(o); + + final TestSubscriber ts = new TestSubscriber(o); + Observable.create(os).subscribeOn(Schedulers.newThread()).subscribe(ts); + + // wait until the first request has started processing + try { + if (!l2.await(1, TimeUnit.SECONDS)) + throw new IllegalStateException(); + } catch (InterruptedException e) {} + // make a concurrent request, this should return + ts.requestMore(count2); + // unblock the 1st thread to proceed fulfilling requests + l1.countDown(); + + ts.awaitTerminalEvent(10, TimeUnit.SECONDS); + ts.assertNoErrors(); + + inOrder.verify(o, times(finalCount)).onNext(any()); + inOrder.verify(o, times(1)).onCompleted(); + inOrder.verifyNoMoreInteractions(); + verify(onUnSubscribe, times(1)).call(any(Integer.class)); + } + + @Test + public void testUnsubscribeOutsideOfLoop() { + final AtomicInteger calledUnsubscribe = new AtomicInteger(0); + final AtomicBoolean currentlyEvaluating = new AtomicBoolean(false); + + OnSubscribe os = SyncOnSubscribe.createStateful( + new Func0() { + @Override + public Void call() { + return null; + }}, + new Func2, Void>() { + @Override + public Void call(Void state, Observer observer) { + currentlyEvaluating.set(true); + observer.onNext(null); + currentlyEvaluating.set(false); + return null; + }}, + new Action1(){ + @Override + public void call(Void t) { + calledUnsubscribe.incrementAndGet(); + assertFalse(currentlyEvaluating.get()); + }}); + + @SuppressWarnings("unchecked") + Observer o = mock(Observer.class); + + final TestSubscriber ts = new TestSubscriber(o) { + @Override + public void onStart() { + requestMore(1); + } + }; + Observable.create(os).lift(new Operator(){ + @Override + public Subscriber call(final Subscriber subscriber) { + return new Subscriber(){ + @Override + public void setProducer(Producer p) { + p.request(1); + } + @Override + public void onCompleted() { + subscriber.onCompleted(); + } + + @Override + public void onError(Throwable e) { + subscriber.onError(e); + } + + @Override + public void onNext(final Void t) { + new Thread(new Runnable(){ + @Override + public void run() { + subscriber.onNext(t); + unsubscribe(); + subscriber.onCompleted(); + }}).start(); + }}; + }}).subscribe(ts); + ts.awaitTerminalEvent(1, TimeUnit.SECONDS); + ts.assertNoErrors(); + ts.assertUnsubscribed(); + assertEquals(1, calledUnsubscribe.get()); + } + + @Test + public void testIndependentStates() { + int count = 100; + final ConcurrentHashMap subscribers = new ConcurrentHashMap(); + + @SuppressWarnings("unchecked") + Action1> onUnSubscribe = mock(Action1.class); + + OnSubscribe os = SyncOnSubscribe.createStateful( + new Func0>() { + @Override + public Map call() { + return subscribers; + }}, + new Func2, Observer, Map>() { + @Override + public Map call(Map state, Observer observer) { + state.put(observer, observer); + observer.onCompleted(); + return state; + }}, + onUnSubscribe); + + Observable source = Observable.create(os); + for (int i = 0; i < count; i++) { + source.subscribe(); + } + + assertEquals(count, subscribers.size()); + verify(onUnSubscribe, times(count)).call(Matchers.>any()); + } + + @Test(timeout = 3000) + public void testSubscribeOn() { + final int start = 1; + final int count = 400; + final AtomicInteger countUnsubscribe = new AtomicInteger(0); + final int numSubscribers = 4; + + OnSubscribe os = SyncOnSubscribe.createStateful( + new Func0() { + @Override + public Integer call() { + return start; + }}, + new Func2, Integer>() { + @Override + public Integer call(Integer calls, Observer observer) { + if (calls > count) { + observer.onCompleted(); + } else { + observer.onNext(calls); + } + return calls + 1; + }}, + new Action1() { + @Override + public void call(Integer t) { + countUnsubscribe.incrementAndGet(); + }}); + + List> subs = new ArrayList>(numSubscribers); + List> mocks = new ArrayList>(numSubscribers); + for (int i = 0; i < numSubscribers; i++) { + @SuppressWarnings("unchecked") + Observer o = mock(Observer.class); + TestSubscriber ts = new TestSubscriber(o); + subs.add(ts); + mocks.add(o); + } + + Observable o2 = Observable.create(os).subscribeOn(Schedulers.newThread()); + for (Subscriber ts : subs) { + o2.subscribe(ts); + } + + for (TestSubscriber ts : subs) { + ts.awaitTerminalEventAndUnsubscribeOnTimeout(1, TimeUnit.SECONDS); + } + + for (Observer o : mocks) { + verify(o, never()).onError(any(Throwable.class)); + verify(o, times(count)).onNext(any()); + verify(o, times(1)).onCompleted(); + } + assertEquals(numSubscribers, countUnsubscribe.get()); + } + + @Test(timeout = 10000) + public void testObserveOn() { + final int start = 1; + final int count = 4000; + + @SuppressWarnings("unchecked") + Action1 onUnSubscribe = mock(Action1.class); + @SuppressWarnings("unchecked") + Func0 generator = mock(Func0.class); + Mockito.when(generator.call()).thenReturn(start); + + OnSubscribe os = SyncOnSubscribe.createStateful(generator, + new Func2, Integer>() { + @Override + public Integer call(Integer calls, Observer observer) { + observer.onNext(calls); + if (calls == count) + observer.onCompleted(); + return calls + 1; + }}, + onUnSubscribe); + + @SuppressWarnings("unchecked") + Observer o = mock(Observer.class); + + TestSubscriber ts = new TestSubscriber(o); + + TestScheduler scheduler = new TestScheduler(); + Observable.create(os).observeOn(scheduler).subscribe(ts); + + scheduler.triggerActions(); + ts.awaitTerminalEvent(); + + verify(o, never()).onError(any(Throwable.class)); + verify(o, times(count)).onNext(any(Integer.class)); + verify(o).onCompleted(); + verify(generator, times(1)).call(); + + List events = ts.getOnNextEvents(); + for (int i = 0; i < events.size(); i++) { + assertEquals(i + 1, events.get(i)); + } + verify(onUnSubscribe, times(1)).call(any(Integer.class)); + } + + @Test + public void testCanRequestInOnNext() { + Action0 onUnSubscribe = mock(Action0.class); + + OnSubscribe os = SyncOnSubscribe.createStateless( + new Action1>() { + @Override + public void call(Observer observer) { + observer.onNext(1); + observer.onCompleted(); + }}, + onUnSubscribe); + final AtomicReference exception = new AtomicReference(); + Observable.create(os).subscribe(new Subscriber() { + @Override + public void onCompleted() { + + } + + @Override + public void onError(Throwable e) { + exception.set(e); + } + + @Override + public void onNext(Integer t) { + request(1); + } + }); + if (exception.get() != null) { + exception.get().printStackTrace(); + } + assertNull(exception.get()); + verify(onUnSubscribe, times(1)).call(); + } + + @Test + public void testExtendingBase() { + final AtomicReference lastState = new AtomicReference(); + final AtomicInteger countUnsubs = new AtomicInteger(0); + SyncOnSubscribe sos = new SyncOnSubscribe() { + @Override + protected Object generateState() { + Object o = new Object(); + lastState.set(o); + return o; + } + + @Override + protected Object next(Object state, Observer observer) { + observer.onNext(lastState.get()); + assertEquals(lastState.get(), state); + Object o = new Object(); + lastState.set(o); + return o; + } + + @Override + protected void onUnsubscribe(Object state) { + countUnsubs.incrementAndGet(); + assertEquals(lastState.get(), state); + } + }; + + @SuppressWarnings("unchecked") + Observer o = mock(Observer.class); + + TestSubscriber ts = new TestSubscriber(o); + + int count = 10; + Observable.create(sos).take(count).subscribe(ts); + + verify(o, never()).onError(any(Throwable.class)); + verify(o, times(count)).onNext(any(Object.class)); + verify(o).onCompleted(); + assertEquals(1, countUnsubs.get()); + } + + private interface FooQux {} + private static class Foo implements FooQux {} + private interface BarQux extends FooQux {} + private static class Bar extends Foo implements BarQux {} + + @Test + public void testGenericsCreateSingleState() { + Func0 generator = new Func0() { + @Override + public Bar call() { + return new Bar(); + }}; + Action2> next = new Action2>() { + @Override + public void call(BarQux state, Observer observer) { + observer.onNext(state); + observer.onCompleted(); + }}; + assertJustBehavior(SyncOnSubscribe.createSingleState(generator, next)); + } + + @Test + public void testGenericsCreateSingleStateWithUnsub() { + Func0 generator = new Func0() { + @Override + public Bar call() { + return new Bar(); + }}; + Action2> next = new Action2>() { + @Override + public void call(BarQux state, Observer observer) { + observer.onNext(state); + observer.onCompleted(); + }}; + Action1 unsub = new Action1() { + @Override + public void call(FooQux t) { + + }}; + assertJustBehavior(SyncOnSubscribe.createSingleState(generator, next, unsub)); + } + + @Test + public void testGenericsCreateStateful() { + Func0 generator = new Func0() { + @Override + public Bar call() { + return new Bar(); + }}; + Func2, ? extends BarQux> next = new Func2, BarQux>() { + @Override + public BarQux call(BarQux state, Observer observer) { + observer.onNext(state); + observer.onCompleted(); + return state; + }}; + assertJustBehavior(SyncOnSubscribe.createStateful(generator, next)); + } + + @Test + public void testGenericsCreateStatefulWithUnsub() { + Func0 generator = new Func0() { + @Override + public Bar call() { + return new Bar(); + }}; + Func2, ? extends BarQux> next = new Func2, BarQux>() { + @Override + public BarQux call(BarQux state, Observer observer) { + observer.onNext(state); + observer.onCompleted(); + return state; + }}; + Action1 unsub = new Action1() { + @Override + public void call(FooQux t) { + + }}; + OnSubscribe os = SyncOnSubscribe.createStateful(generator, next, unsub); + assertJustBehavior(os); + } + + @Test + public void testGenericsCreateStateless() { + Action1> next = new Action1>() { + @Override + public void call(Observer observer) { + observer.onNext(new Foo()); + observer.onCompleted(); + }}; + OnSubscribe os = SyncOnSubscribe.createStateless(next); + assertJustBehavior(os); + } + + @Test + public void testGenericsCreateStatelessWithUnsub() { + Action1> next = new Action1>() { + @Override + public void call(Observer observer) { + observer.onNext(new Foo()); + observer.onCompleted(); + }}; + Action0 unsub = new Action0() { + @Override + public void call() { + + }}; + OnSubscribe os = SyncOnSubscribe.createStateless(next, unsub); + assertJustBehavior(os); + } + + private void assertJustBehavior(OnSubscribe os) { + @SuppressWarnings("unchecked") + Observer o = mock(Observer.class); + + TestSubscriber ts = new TestSubscriber(o); + + os.call(ts); + verify(o, times(1)).onNext(any()); + verify(o, times(1)).onCompleted(); + verify(o, never()).onError(any(Throwable.class)); + } +} From 55fcd41073e7a56ef5651984014ee7b0ac08a862 Mon Sep 17 00:00:00 2001 From: Aaron Tull Date: Wed, 2 Sep 2015 12:16:03 -0700 Subject: [PATCH 50/81] Fixing concurrent unsubscribe case of SyncOnSubscribe --- .../java/rx/observables/SyncOnSubscribe.java | 74 +++--- .../rx/observables/SyncOnSubscribeTest.java | 214 ++++++++++-------- 2 files changed, 156 insertions(+), 132 deletions(-) diff --git a/src/main/java/rx/observables/SyncOnSubscribe.java b/src/main/java/rx/observables/SyncOnSubscribe.java index 47a6c34024..c75173a094 100644 --- a/src/main/java/rx/observables/SyncOnSubscribe.java +++ b/src/main/java/rx/observables/SyncOnSubscribe.java @@ -16,7 +16,6 @@ package rx.observables; -import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.concurrent.atomic.AtomicLong; import rx.Observable.OnSubscribe; @@ -321,14 +320,9 @@ private static class SubscriptionProducer private final SyncOnSubscribe parent; private boolean onNextCalled; private boolean hasTerminated; - + private S state; - volatile int isUnsubscribed; - @SuppressWarnings("rawtypes") - static final AtomicIntegerFieldUpdater IS_UNSUBSCRIBED = - AtomicIntegerFieldUpdater.newUpdater(SubscriptionProducer.class, "isUnsubscribed"); - private SubscriptionProducer(final Subscriber subscriber, SyncOnSubscribe parent, S state) { this.actualSubscriber = subscriber; this.parent = parent; @@ -337,14 +331,39 @@ private SubscriptionProducer(final Subscriber subscriber, SyncOnSubsc @Override public boolean isUnsubscribed() { - return isUnsubscribed != 0; + return get() < 0L; } @Override public void unsubscribe() { - IS_UNSUBSCRIBED.compareAndSet(this, 0, 1); - if (get() == 0L) - parent.onUnsubscribe(state); + while(true) { + long requestCount = get(); + if (compareAndSet(0L, -1L)) { + doUnsubscribe(); + return; + } + else if (compareAndSet(requestCount, -2L)) + // the loop is iterating concurrently + // need to check if requestCount == -1 + // and unsub if so after loop iteration + return; + } + } + + private boolean tryUnsubscribe() { + // only one thread at a time can iterate over request count + // therefore the requestCount atomic cannot be decrement concurrently here + // safe to set to -1 atomically (since this check can only be done by 1 thread) + if (hasTerminated || get() < -1) { + set(-1); + doUnsubscribe(); + return true; + } + return false; + } + + private void doUnsubscribe() { + parent.onUnsubscribe(state); } @Override @@ -358,71 +377,60 @@ public void request(long n) { } } - void fastpath() { + private void fastpath() { final SyncOnSubscribe p = parent; Subscriber a = actualSubscriber; - if (isUnsubscribed()) { - p.onUnsubscribe(state); - return; - } - for (;;) { try { onNextCalled = false; nextIteration(p); } catch (Throwable ex) { - handleThrownError(p, a, state, ex); + handleThrownError(a, ex); return; } - if (hasTerminated || isUnsubscribed()) { - p.onUnsubscribe(state); + if (tryUnsubscribe()) { return; } } } - private void handleThrownError(final SyncOnSubscribe p, Subscriber a, S st, Throwable ex) { + private void handleThrownError(Subscriber a, Throwable ex) { if (hasTerminated) { RxJavaPlugins.getInstance().getErrorHandler().handleError(ex); } else { hasTerminated = true; a.onError(ex); - p.onUnsubscribe(st); + unsubscribe(); } } - void slowPath(long n) { + private void slowPath(long n) { final SyncOnSubscribe p = parent; Subscriber a = actualSubscriber; long numRequested = n; for (;;) { - if (isUnsubscribed()) { - p.onUnsubscribe(state); - return; - } long numRemaining = numRequested; do { try { onNextCalled = false; nextIteration(p); } catch (Throwable ex) { - handleThrownError(p, a, state, ex); + handleThrownError(a, ex); return; } - if (hasTerminated || isUnsubscribed()) { - p.onUnsubscribe(state); + if (tryUnsubscribe()) { return; } if (onNextCalled) numRemaining--; } while (numRemaining != 0L); - numRequested = addAndGet(-numRequested); - if (numRequested == 0L) { + if (numRequested <= 0L) break; - } } + // catches cases where unsubscribe is called before decrementing atomic request count + tryUnsubscribe(); } private void nextIteration(final SyncOnSubscribe parent) { diff --git a/src/test/java/rx/observables/SyncOnSubscribeTest.java b/src/test/java/rx/observables/SyncOnSubscribeTest.java index 91421d502a..22e1f11cfd 100644 --- a/src/test/java/rx/observables/SyncOnSubscribeTest.java +++ b/src/test/java/rx/observables/SyncOnSubscribeTest.java @@ -17,9 +17,9 @@ package rx.observables; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertFalse; import static org.mockito.Matchers.any; import static org.mockito.Matchers.isA; import static org.mockito.Mockito.inOrder; @@ -33,8 +33,15 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.concurrent.BrokenBarrierException; +import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -65,6 +72,7 @@ * Test if SyncOnSubscribe adheres to the usual unsubscription and backpressure contracts. */ public class SyncOnSubscribeTest { + @Test public void testObservableJustEquivalent() { OnSubscribe os = SyncOnSubscribe.createStateless(new Action1>() { @@ -91,13 +99,14 @@ public void testStateAfterTerminal() { public Integer call() { return 1; }}, - new Func2, Integer>() { - @Override - public Integer call(Integer state, Observer subscriber) { - subscriber.onNext(state); - subscriber.onCompleted(); - return state + 1; - }}, new Action1() { + new Func2, Integer>() { + @Override + public Integer call(Integer state, Observer subscriber) { + subscriber.onNext(state); + subscriber.onCompleted(); + return state + 1; + }}, + new Action1() { @Override public void call(Integer t) { finalStateValue.set(t); @@ -438,25 +447,14 @@ public Integer call(Integer state, Observer observer) { }}, onUnSubscribe); - @SuppressWarnings("unchecked") - Observer o = mock(Observer.class); - InOrder inOrder = inOrder(o); - - TestSubscriber ts = new TestSubscriber(o) { - @Override - public void onStart() { - requestMore(0); // don't start right away - } - }; - + TestSubscriber ts = new TestSubscriber(0); Observable.create(os).subscribe(ts); ts.requestMore(finalCount); - verify(o, never()).onError(any(Throwable.class)); - verify(o, never()).onCompleted(); - inOrder.verify(o, times(finalCount)).onNext(any()); - inOrder.verifyNoMoreInteractions(); + ts.assertNoErrors(); + ts.assertNotCompleted(); + ts.assertValueCount(finalCount); // unsubscribe does not take place because subscriber is still in process of requesting verify(onUnSubscribe, never()).call(any(Integer.class)); } @@ -485,31 +483,7 @@ public Integer call(Integer state, Observer observer) { TestSubscriber ts = new TestSubscriber(o); - Observable.create(os).lift(new Operator(){ - @Override - public Subscriber call(final Subscriber subscriber) { - return new Subscriber(){ - @Override - public void setProducer(Producer p) { - p.request(Long.MAX_VALUE); - } - - @Override - public void onCompleted() { - subscriber.onCompleted(); - } - - @Override - public void onError(Throwable e) { - subscriber.onError(e); - } - - @Override - public void onNext(Object t) { - subscriber.onNext(t); - unsubscribe(); - }}; - }}).take(1).subscribe(ts); + Observable.create(os).take(1).subscribe(ts); verify(o, never()).onError(any(Throwable.class)); verify(onUnSubscribe, times(1)).call(any(Integer.class)); @@ -577,27 +551,21 @@ public Integer call(Integer state, Observer observer) { } @Test - public void testUnsubscribeOutsideOfLoop() { + public void testUnsubscribeOutsideOfLoop() throws InterruptedException { final AtomicInteger calledUnsubscribe = new AtomicInteger(0); final AtomicBoolean currentlyEvaluating = new AtomicBoolean(false); - OnSubscribe os = SyncOnSubscribe.createStateful( - new Func0() { - @Override - public Void call() { - return null; - }}, - new Func2, Void>() { + OnSubscribe os = SyncOnSubscribe.createStateless( + new Action1>() { @Override - public Void call(Void state, Observer observer) { + public void call(Observer observer) { currentlyEvaluating.set(true); observer.onNext(null); currentlyEvaluating.set(false); - return null; }}, - new Action1(){ + new Action0(){ @Override - public void call(Void t) { + public void call() { calledUnsubscribe.incrementAndGet(); assertFalse(currentlyEvaluating.get()); }}); @@ -605,16 +573,12 @@ public void call(Void t) { @SuppressWarnings("unchecked") Observer o = mock(Observer.class); - final TestSubscriber ts = new TestSubscriber(o) { - @Override - public void onStart() { - requestMore(1); - } - }; + final CountDownLatch latch = new CountDownLatch(1); + final TestSubscriber ts = new TestSubscriber(o); Observable.create(os).lift(new Operator(){ @Override public Subscriber call(final Subscriber subscriber) { - return new Subscriber(){ + return new Subscriber(subscriber){ @Override public void setProducer(Producer p) { p.request(1); @@ -631,16 +595,23 @@ public void onError(Throwable e) { @Override public void onNext(final Void t) { + subscriber.onNext(t); new Thread(new Runnable(){ @Override public void run() { - subscriber.onNext(t); + try { + latch.await(1, TimeUnit.SECONDS); + } catch (InterruptedException e) { + e.printStackTrace(); + } unsubscribe(); subscriber.onCompleted(); + latch.countDown(); }}).start(); }}; }}).subscribe(ts); - ts.awaitTerminalEvent(1, TimeUnit.SECONDS); + latch.countDown(); + ts.awaitTerminalEventAndUnsubscribeOnTimeout(1, TimeUnit.SECONDS); ts.assertNoErrors(); ts.assertUnsubscribed(); assertEquals(1, calledUnsubscribe.get()); @@ -708,29 +679,23 @@ public void call(Integer t) { }}); List> subs = new ArrayList>(numSubscribers); - List> mocks = new ArrayList>(numSubscribers); for (int i = 0; i < numSubscribers; i++) { - @SuppressWarnings("unchecked") - Observer o = mock(Observer.class); - TestSubscriber ts = new TestSubscriber(o); + TestSubscriber ts = new TestSubscriber(); subs.add(ts); - mocks.add(o); } - - Observable o2 = Observable.create(os).subscribeOn(Schedulers.newThread()); + TestScheduler scheduler = new TestScheduler(); + Observable o2 = Observable.create(os).subscribeOn(scheduler); for (Subscriber ts : subs) { o2.subscribe(ts); } - + scheduler.triggerActions(); for (TestSubscriber ts : subs) { - ts.awaitTerminalEventAndUnsubscribeOnTimeout(1, TimeUnit.SECONDS); + ts.awaitTerminalEvent(1, TimeUnit.SECONDS); + ts.assertNoErrors(); + ts.assertValueCount(count); + ts.assertCompleted(); } - for (Observer o : mocks) { - verify(o, never()).onError(any(Throwable.class)); - verify(o, times(count)).onNext(any()); - verify(o, times(1)).onCompleted(); - } assertEquals(numSubscribers, countUnsubscribe.get()); } @@ -756,20 +721,16 @@ public Integer call(Integer calls, Observer observer) { }}, onUnSubscribe); - @SuppressWarnings("unchecked") - Observer o = mock(Observer.class); - - TestSubscriber ts = new TestSubscriber(o); + TestSubscriber ts = new TestSubscriber(); TestScheduler scheduler = new TestScheduler(); Observable.create(os).observeOn(scheduler).subscribe(ts); scheduler.triggerActions(); ts.awaitTerminalEvent(); - - verify(o, never()).onError(any(Throwable.class)); - verify(o, times(count)).onNext(any(Integer.class)); - verify(o).onCompleted(); + ts.assertNoErrors(); + ts.assertCompleted(); + ts.assertValueCount(count); verify(generator, times(1)).call(); List events = ts.getOnNextEvents(); @@ -969,14 +930,69 @@ public void call() { } private void assertJustBehavior(OnSubscribe os) { - @SuppressWarnings("unchecked") - Observer o = mock(Observer.class); - - TestSubscriber ts = new TestSubscriber(o); - + TestSubscriber ts = new TestSubscriber(); os.call(ts); - verify(o, times(1)).onNext(any()); - verify(o, times(1)).onCompleted(); - verify(o, never()).onError(any(Throwable.class)); + ts.assertCompleted(); + ts.assertNoErrors(); + ts.assertValueCount(1); + } + + @Test + public void testConcurrentUnsubscribe3000Iterations() throws InterruptedException, BrokenBarrierException, ExecutionException{ + ExecutorService exec = null; + try { + exec = Executors.newSingleThreadExecutor(); + for (int i = 0; i < 3000; i++) { + final AtomicInteger wip = new AtomicInteger(); + + Func0 func0 = new Func0() { + @Override + public AtomicInteger call() { + return wip; + } + }; + Func2, AtomicInteger> func2 = + new Func2, AtomicInteger>() { + @Override + public AtomicInteger call(AtomicInteger s, Observer o) { + o.onNext(1); + return s; + } + }; + Action1 action1 = new Action1() { + @Override + public void call(AtomicInteger s) { + s.getAndIncrement(); + } + }; + Observable source = Observable.create( + SyncOnSubscribe.createStateful( + func0, + func2, action1 + )); + + + final TestSubscriber ts = TestSubscriber.create(0); + source.subscribe(ts); + + final CyclicBarrier cb = new CyclicBarrier(2); + + Future f = exec.submit(new Callable() { + @Override + public Object call() throws Exception { + cb.await(); + ts.requestMore(1); + return null; + } + }); + + cb.await(); + ts.unsubscribe(); + f.get(); + assertEquals("Unsubscribe supposed to be called once", 1, wip.get()); + } + } finally { + if (exec != null) exec.shutdownNow(); + } } } From f2a840089f86af82ac4cfc22a6028d2f52e591e7 Mon Sep 17 00:00:00 2001 From: Artem Zinnatullin Date: Fri, 14 Aug 2015 03:25:14 +0300 Subject: [PATCH 51/81] Add Observable.fromCallable() as a companion for Observable.defer() --- src/main/java/rx/Observable.java | 23 +++ .../operators/OnSubscribeFromCallable.java | 38 +++++ .../OnSubscribeFromCallableTest.java | 140 ++++++++++++++++++ 3 files changed, 201 insertions(+) create mode 100644 src/main/java/rx/internal/operators/OnSubscribeFromCallable.java create mode 100644 src/test/java/rx/internal/operators/OnSubscribeFromCallableTest.java diff --git a/src/main/java/rx/Observable.java b/src/main/java/rx/Observable.java index 79825d622b..5d6d77c15f 100644 --- a/src/main/java/rx/Observable.java +++ b/src/main/java/rx/Observable.java @@ -1250,6 +1250,29 @@ public final static Observable from(T[] array) { return from(Arrays.asList(array)); } + /** + * Returns an Observable that invokes passed function and emits its result for each new Observer that subscribes. + *

+ * Allows you to defer execution of passed function until Observer subscribes to the Observable. + * It makes passed function "lazy". + * Result of the function invocation will be emitted by the Observable. + *

+ *
Scheduler:
+ *
{@code fromCallable} does not operate by default on a particular {@link Scheduler}.
+ *
+ * + * @param func + * function which execution should be deferred, it will be invoked when Observer will subscribe to the Observable + * @param + * the type of the item emitted by the Observable + * @return an Observable whose {@link Observer}s' subscriptions trigger an invocation of the given function + * @see #defer(Func0) + */ + @Experimental + public static Observable fromCallable(Callable func) { + return create(new OnSubscribeFromCallable(func)); + } + /** * Returns an Observable that emits a sequential number every specified interval of time. *

diff --git a/src/main/java/rx/internal/operators/OnSubscribeFromCallable.java b/src/main/java/rx/internal/operators/OnSubscribeFromCallable.java new file mode 100644 index 0000000000..35eb62f04e --- /dev/null +++ b/src/main/java/rx/internal/operators/OnSubscribeFromCallable.java @@ -0,0 +1,38 @@ +package rx.internal.operators; + +import rx.Observable; +import rx.Subscriber; +import rx.exceptions.Exceptions; +import rx.internal.producers.SingleDelayedProducer; + +import java.util.concurrent.Callable; + +/** + * Do not invoke the function until an Observer subscribes; Invokes function on each + * subscription. + *

+ * Pass {@code fromCallable} a function, and {@code fromCallable} will call this function to emit result of invocation + * afresh each time a new Observer subscribes. + */ +public final class OnSubscribeFromCallable implements Observable.OnSubscribe { + + private final Callable resultFactory; + + public OnSubscribeFromCallable(Callable resultFactory) { + this.resultFactory = resultFactory; + } + + @Override + public void call(Subscriber subscriber) { + final SingleDelayedProducer singleDelayedProducer = new SingleDelayedProducer(subscriber); + + subscriber.setProducer(singleDelayedProducer); + + try { + singleDelayedProducer.setValue(resultFactory.call()); + } catch (Throwable t) { + Exceptions.throwIfFatal(t); + subscriber.onError(t); + } + } +} diff --git a/src/test/java/rx/internal/operators/OnSubscribeFromCallableTest.java b/src/test/java/rx/internal/operators/OnSubscribeFromCallableTest.java new file mode 100644 index 0000000000..a4da6e3208 --- /dev/null +++ b/src/test/java/rx/internal/operators/OnSubscribeFromCallableTest.java @@ -0,0 +1,140 @@ +package rx.internal.operators; + +import org.junit.Test; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; +import rx.Observable; +import rx.Observer; +import rx.Subscription; + +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; + +import static org.mockito.Mockito.*; +import static rx.schedulers.Schedulers.computation; + +public class OnSubscribeFromCallableTest { + + @SuppressWarnings("unchecked") + @Test + public void shouldNotInvokeFuncUntilSubscription() throws Exception { + Callable func = mock(Callable.class); + + when(func.call()).thenReturn(new Object()); + + Observable fromCallableObservable = Observable.fromCallable(func); + + verifyZeroInteractions(func); + + fromCallableObservable.subscribe(); + + verify(func).call(); + } + + @SuppressWarnings("unchecked") + @Test + public void shouldCallOnNextAndOnCompleted() throws Exception { + Callable func = mock(Callable.class); + + when(func.call()).thenReturn("test_value"); + + Observable fromCallableObservable = Observable.fromCallable(func); + + Observer observer = mock(Observer.class); + + fromCallableObservable.subscribe(observer); + + verify(observer).onNext("test_value"); + verify(observer).onCompleted(); + verify(observer, never()).onError(any(Throwable.class)); + } + + @SuppressWarnings("unchecked") + @Test + public void shouldCallOnError() throws Exception { + Callable func = mock(Callable.class); + + Throwable throwable = new IllegalStateException("Test exception"); + when(func.call()).thenThrow(throwable); + + Observable fromCallableObservable = Observable.fromCallable(func); + + Observer observer = mock(Observer.class); + + fromCallableObservable.subscribe(observer); + + verify(observer, never()).onNext(anyObject()); + verify(observer, never()).onCompleted(); + verify(observer).onError(throwable); + } + + @SuppressWarnings("unchecked") + @Test + public void shouldNotDeliverResultIfSubscriberUnsubscribedBeforeEmission() throws Exception { + Callable func = mock(Callable.class); + + final CountDownLatch funcLatch = new CountDownLatch(1); + final CountDownLatch observerLatch = new CountDownLatch(1); + + when(func.call()).thenAnswer(new Answer() { + @Override + public String answer(InvocationOnMock invocation) throws Throwable { + observerLatch.countDown(); + + try { + funcLatch.await(); + } catch (InterruptedException e) { + // It's okay, unsubscription causes Thread interruption + + // Restoring interruption status of the Thread + Thread.currentThread().interrupt(); + } + + return "should_not_be_delivered"; + } + }); + + Observable fromCallableObservable = Observable.fromCallable(func); + + Observer observer = mock(Observer.class); + + Subscription subscription = fromCallableObservable + .subscribeOn(computation()) + .subscribe(observer); + + // Wait until func will be invoked + observerLatch.await(); + + // Unsubscribing before emission + subscription.unsubscribe(); + + // Emitting result + funcLatch.countDown(); + + // func must be invoked + verify(func).call(); + + // Observer must not be notified at all + verifyZeroInteractions(observer); + } + + @SuppressWarnings("unchecked") + @Test + public void shouldAllowToThrowCheckedException() { + final Exception checkedException = new Exception("test exception"); + + Observable fromCallableObservable = Observable.fromCallable(new Callable() { + @Override + public Object call() throws Exception { + throw checkedException; + } + }); + + Observer observer = mock(Observer.class); + + fromCallableObservable.subscribe(observer); + + verify(observer).onError(checkedException); + verifyNoMoreInteractions(observer); + } +} \ No newline at end of file From 2df3c6481a21487dd28d179de49aaeea711c2861 Mon Sep 17 00:00:00 2001 From: Tobias Bieniek Date: Fri, 11 Sep 2015 14:05:34 +0200 Subject: [PATCH 52/81] test/subjects: Use statically imported never() methods ... and remove the unused Mockito imports --- .../java/rx/subjects/AsyncSubjectTest.java | 33 +++++++++---------- .../java/rx/subjects/BehaviorSubjectTest.java | 15 ++++----- .../java/rx/subjects/PublishSubjectTest.java | 17 +++++----- .../java/rx/subjects/ReplaySubjectTest.java | 20 +++++------ 4 files changed, 40 insertions(+), 45 deletions(-) diff --git a/src/test/java/rx/subjects/AsyncSubjectTest.java b/src/test/java/rx/subjects/AsyncSubjectTest.java index c4287fc363..623cdceb3f 100644 --- a/src/test/java/rx/subjects/AsyncSubjectTest.java +++ b/src/test/java/rx/subjects/AsyncSubjectTest.java @@ -33,7 +33,6 @@ import org.junit.Test; import org.mockito.InOrder; -import org.mockito.Mockito; import rx.Observer; import rx.Subscription; @@ -59,9 +58,9 @@ public void testNeverCompleted() { subject.onNext("two"); subject.onNext("three"); - verify(observer, Mockito.never()).onNext(anyString()); - verify(observer, Mockito.never()).onError(testException); - verify(observer, Mockito.never()).onCompleted(); + verify(observer, never()).onNext(anyString()); + verify(observer, never()).onError(testException); + verify(observer, never()).onCompleted(); } @Test @@ -78,7 +77,7 @@ public void testCompleted() { subject.onCompleted(); verify(observer, times(1)).onNext("three"); - verify(observer, Mockito.never()).onError(any(Throwable.class)); + verify(observer, never()).onError(any(Throwable.class)); verify(observer, times(1)).onCompleted(); } @@ -94,7 +93,7 @@ public void testNull() { subject.onCompleted(); verify(observer, times(1)).onNext(null); - verify(observer, Mockito.never()).onError(any(Throwable.class)); + verify(observer, never()).onError(any(Throwable.class)); verify(observer, times(1)).onCompleted(); } @@ -113,7 +112,7 @@ public void testSubscribeAfterCompleted() { subject.subscribe(observer); verify(observer, times(1)).onNext("three"); - verify(observer, Mockito.never()).onError(any(Throwable.class)); + verify(observer, never()).onError(any(Throwable.class)); verify(observer, times(1)).onCompleted(); } @@ -134,8 +133,8 @@ public void testSubscribeAfterError() { subject.subscribe(observer); verify(observer, times(1)).onError(re); - verify(observer, Mockito.never()).onNext(any(String.class)); - verify(observer, Mockito.never()).onCompleted(); + verify(observer, never()).onNext(any(String.class)); + verify(observer, never()).onCompleted(); } @Test @@ -154,9 +153,9 @@ public void testError() { subject.onError(new Throwable()); subject.onCompleted(); - verify(observer, Mockito.never()).onNext(anyString()); + verify(observer, never()).onNext(anyString()); verify(observer, times(1)).onError(testException); - verify(observer, Mockito.never()).onCompleted(); + verify(observer, never()).onCompleted(); } @Test @@ -172,16 +171,16 @@ public void testUnsubscribeBeforeCompleted() { subscription.unsubscribe(); - verify(observer, Mockito.never()).onNext(anyString()); - verify(observer, Mockito.never()).onError(any(Throwable.class)); - verify(observer, Mockito.never()).onCompleted(); + verify(observer, never()).onNext(anyString()); + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, never()).onCompleted(); subject.onNext("three"); subject.onCompleted(); - verify(observer, Mockito.never()).onNext(anyString()); - verify(observer, Mockito.never()).onError(any(Throwable.class)); - verify(observer, Mockito.never()).onCompleted(); + verify(observer, never()).onNext(anyString()); + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, never()).onCompleted(); } @Test diff --git a/src/test/java/rx/subjects/BehaviorSubjectTest.java b/src/test/java/rx/subjects/BehaviorSubjectTest.java index c64afa4efb..f577fa7c37 100644 --- a/src/test/java/rx/subjects/BehaviorSubjectTest.java +++ b/src/test/java/rx/subjects/BehaviorSubjectTest.java @@ -32,7 +32,6 @@ import org.junit.*; import org.mockito.InOrder; -import org.mockito.Mockito; import rx.*; import rx.exceptions.CompositeException; @@ -62,8 +61,8 @@ public void testThatObserverReceivesDefaultValueAndSubsequentEvents() { verify(observer, times(1)).onNext("one"); verify(observer, times(1)).onNext("two"); verify(observer, times(1)).onNext("three"); - verify(observer, Mockito.never()).onError(testException); - verify(observer, Mockito.never()).onCompleted(); + verify(observer, never()).onError(testException); + verify(observer, never()).onCompleted(); } @Test @@ -79,12 +78,12 @@ public void testThatObserverReceivesLatestAndThenSubsequentEvents() { subject.onNext("two"); subject.onNext("three"); - verify(observer, Mockito.never()).onNext("default"); + verify(observer, never()).onNext("default"); verify(observer, times(1)).onNext("one"); verify(observer, times(1)).onNext("two"); verify(observer, times(1)).onNext("three"); - verify(observer, Mockito.never()).onError(testException); - verify(observer, Mockito.never()).onCompleted(); + verify(observer, never()).onError(testException); + verify(observer, never()).onCompleted(); } @Test @@ -100,7 +99,7 @@ public void testSubscribeThenOnComplete() { verify(observer, times(1)).onNext("default"); verify(observer, times(1)).onNext("one"); - verify(observer, Mockito.never()).onError(any(Throwable.class)); + verify(observer, never()).onError(any(Throwable.class)); verify(observer, times(1)).onCompleted(); } @@ -116,7 +115,7 @@ public void testSubscribeToCompletedOnlyEmitsOnComplete() { verify(observer, never()).onNext("default"); verify(observer, never()).onNext("one"); - verify(observer, Mockito.never()).onError(any(Throwable.class)); + verify(observer, never()).onError(any(Throwable.class)); verify(observer, times(1)).onCompleted(); } diff --git a/src/test/java/rx/subjects/PublishSubjectTest.java b/src/test/java/rx/subjects/PublishSubjectTest.java index f8bc989112..44fe824a5c 100644 --- a/src/test/java/rx/subjects/PublishSubjectTest.java +++ b/src/test/java/rx/subjects/PublishSubjectTest.java @@ -32,7 +32,6 @@ import org.junit.Test; import org.mockito.InOrder; -import org.mockito.Mockito; import rx.Observable; import rx.Observer; @@ -118,7 +117,7 @@ private void assertCompletedObserver(Observer observer) { verify(observer, times(1)).onNext("one"); verify(observer, times(1)).onNext("two"); verify(observer, times(1)).onNext("three"); - verify(observer, Mockito.never()).onError(any(Throwable.class)); + verify(observer, never()).onError(any(Throwable.class)); verify(observer, times(1)).onCompleted(); } @@ -152,7 +151,7 @@ private void assertErrorObserver(Observer observer) { verify(observer, times(1)).onNext("two"); verify(observer, times(1)).onNext("three"); verify(observer, times(1)).onError(testException); - verify(observer, Mockito.never()).onCompleted(); + verify(observer, never()).onCompleted(); } @Test @@ -180,10 +179,10 @@ public void testSubscribeMidSequence() { } private void assertCompletedStartingWithThreeObserver(Observer observer) { - verify(observer, Mockito.never()).onNext("one"); - verify(observer, Mockito.never()).onNext("two"); + verify(observer, never()).onNext("one"); + verify(observer, never()).onNext("two"); verify(observer, times(1)).onNext("three"); - verify(observer, Mockito.never()).onError(any(Throwable.class)); + verify(observer, never()).onError(any(Throwable.class)); verify(observer, times(1)).onCompleted(); } @@ -215,9 +214,9 @@ public void testUnsubscribeFirstObserver() { private void assertObservedUntilTwo(Observer observer) { verify(observer, times(1)).onNext("one"); verify(observer, times(1)).onNext("two"); - verify(observer, Mockito.never()).onNext("three"); - verify(observer, Mockito.never()).onError(any(Throwable.class)); - verify(observer, Mockito.never()).onCompleted(); + verify(observer, never()).onNext("three"); + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, never()).onCompleted(); } @Test diff --git a/src/test/java/rx/subjects/ReplaySubjectTest.java b/src/test/java/rx/subjects/ReplaySubjectTest.java index bd01901bc1..5ebb871604 100644 --- a/src/test/java/rx/subjects/ReplaySubjectTest.java +++ b/src/test/java/rx/subjects/ReplaySubjectTest.java @@ -36,7 +36,6 @@ import org.junit.Test; import org.mockito.InOrder; -import org.mockito.Mockito; import rx.Observable; import rx.Observer; @@ -140,11 +139,10 @@ public void testCompletedStopsEmittingData() { inOrderD.verify(observerD).onNext(4711); inOrderD.verify(observerD).onCompleted(); - Mockito.verifyNoMoreInteractions(observerA); - Mockito.verifyNoMoreInteractions(observerB); - Mockito.verifyNoMoreInteractions(observerC); - Mockito.verifyNoMoreInteractions(observerD); - + verifyNoMoreInteractions(observerA); + verifyNoMoreInteractions(observerB); + verifyNoMoreInteractions(observerC); + verifyNoMoreInteractions(observerD); } @Test @@ -172,7 +170,7 @@ private void assertCompletedObserver(Observer observer) { inOrder.verify(observer, times(1)).onNext("one"); inOrder.verify(observer, times(1)).onNext("two"); inOrder.verify(observer, times(1)).onNext("three"); - inOrder.verify(observer, Mockito.never()).onError(any(Throwable.class)); + inOrder.verify(observer, never()).onError(any(Throwable.class)); inOrder.verify(observer, times(1)).onCompleted(); inOrder.verifyNoMoreInteractions(); } @@ -206,7 +204,7 @@ private void assertErrorObserver(Observer observer) { verify(observer, times(1)).onNext("two"); verify(observer, times(1)).onNext("three"); verify(observer, times(1)).onError(testException); - verify(observer, Mockito.never()).onCompleted(); + verify(observer, never()).onCompleted(); } @SuppressWarnings("unchecked") @@ -261,9 +259,9 @@ public void testUnsubscribeFirstObserver() { private void assertObservedUntilTwo(Observer observer) { verify(observer, times(1)).onNext("one"); verify(observer, times(1)).onNext("two"); - verify(observer, Mockito.never()).onNext("three"); - verify(observer, Mockito.never()).onError(any(Throwable.class)); - verify(observer, Mockito.never()).onCompleted(); + verify(observer, never()).onNext("three"); + verify(observer, never()).onError(any(Throwable.class)); + verify(observer, never()).onCompleted(); } @Test(timeout = 2000) From 9cc4b1ba8b67d83d2c8a52d222a64520b5d497f4 Mon Sep 17 00:00:00 2001 From: Tobias Bieniek Date: Fri, 11 Sep 2015 14:23:21 +0200 Subject: [PATCH 53/81] BehaviorSubjectTest: Fix verification in testCompletedAfterErrorIsNotSent3() --- src/test/java/rx/subjects/BehaviorSubjectTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/java/rx/subjects/BehaviorSubjectTest.java b/src/test/java/rx/subjects/BehaviorSubjectTest.java index f577fa7c37..e520970d12 100644 --- a/src/test/java/rx/subjects/BehaviorSubjectTest.java +++ b/src/test/java/rx/subjects/BehaviorSubjectTest.java @@ -253,7 +253,7 @@ public void testCompletedAfterErrorIsNotSent3() { subject.subscribe(o2); verify(o2, times(1)).onCompleted(); verify(o2, never()).onNext(any()); - verify(observer, never()).onError(any(Throwable.class)); + verify(o2, never()).onError(any(Throwable.class)); } @Test(timeout = 1000) public void testUnsubscriptionCase() { From eb945d89ffd30af2627017c38b49dd4c48a86b2c Mon Sep 17 00:00:00 2001 From: Tobias Bieniek Date: Fri, 11 Sep 2015 14:27:17 +0200 Subject: [PATCH 54/81] BehaviorSubjectTest: Simplify testUnsubscriptionCase() test --- .../java/rx/subjects/BehaviorSubjectTest.java | 16 +--------------- 1 file changed, 1 insertion(+), 15 deletions(-) diff --git a/src/test/java/rx/subjects/BehaviorSubjectTest.java b/src/test/java/rx/subjects/BehaviorSubjectTest.java index e520970d12..9e9e4c90e7 100644 --- a/src/test/java/rx/subjects/BehaviorSubjectTest.java +++ b/src/test/java/rx/subjects/BehaviorSubjectTest.java @@ -274,22 +274,8 @@ public Observable call(String t1) { return Observable.just(t1 + ", " + t1); } }) - .subscribe(new Observer() { - @Override - public void onNext(String t) { - o.onNext(t); - } - - @Override - public void onError(Throwable e) { - o.onError(e); - } + .subscribe(o); - @Override - public void onCompleted() { - o.onCompleted(); - } - }); inOrder.verify(o).onNext(v + ", " + v); inOrder.verify(o).onCompleted(); verify(o, never()).onError(any(Throwable.class)); From 1e56564bd928a5cc589657ac62b615b420094566 Mon Sep 17 00:00:00 2001 From: Aaron Tull Date: Wed, 26 Aug 2015 17:48:11 -0700 Subject: [PATCH 55/81] Implemented the AsyncOnSubscribe --- .../java/rx/observables/AsyncOnSubscribe.java | 510 ++++++++++++++++++ .../rx/observables/AsyncOnSubscribeTest.java | 408 ++++++++++++++ 2 files changed, 918 insertions(+) create mode 100644 src/main/java/rx/observables/AsyncOnSubscribe.java create mode 100644 src/test/java/rx/observables/AsyncOnSubscribeTest.java diff --git a/src/main/java/rx/observables/AsyncOnSubscribe.java b/src/main/java/rx/observables/AsyncOnSubscribe.java new file mode 100644 index 0000000000..d4a12b0245 --- /dev/null +++ b/src/main/java/rx/observables/AsyncOnSubscribe.java @@ -0,0 +1,510 @@ +/** + * Copyright 2015 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.observables; + +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import java.util.concurrent.atomic.AtomicReference; + +import rx.Observable; +import rx.Observable.OnSubscribe; +import rx.Observer; +import rx.Producer; +import rx.Subscriber; +import rx.Subscription; +import rx.annotations.Experimental; +import rx.functions.Action0; +import rx.functions.Action1; +import rx.functions.Action2; +import rx.functions.Action3; +import rx.functions.Func0; +import rx.functions.Func3; +import rx.internal.operators.BufferUntilSubscriber; +import rx.observers.SerializedObserver; +import rx.observers.Subscribers; +import rx.plugins.RxJavaPlugins; +import rx.subscriptions.BooleanSubscription; +; +/** + * A utility class to create {@code OnSubscribe} functions that respond correctly to back + * pressure requests from subscribers. This is an improvement over + * {@link rx.Observable#create(OnSubscribe) Observable.create(OnSubscribe)} which does not provide + * any means of managing back pressure requests out-of-the-box. This variant of an OnSubscribe + * function allows for the asynchronous processing of requests. + * + * @param + * the type of the user-define state used in {@link #generateState() generateState(S)} , + * {@link #next(Object, Long, Subscriber) next(S, Long, Subscriber)}, and + * {@link #onUnsubscribe(Object) onUnsubscribe(S)}. + * @param + * the type of {@code Subscribers} that will be compatible with {@code this}. + */ +@Experimental +public abstract class AsyncOnSubscribe implements OnSubscribe { + + /** + * Executed once when subscribed to by a subscriber (via {@link OnSubscribe#call(Subscriber)}) + * to produce a state value. This value is passed into {@link #next(Object, long, Observer) + * next(S state, Observer observer)} on the first iteration. Subsequent iterations of + * {@code next} will receive the state returned by the previous invocation of {@code next}. + * + * @return the initial state value + */ + protected abstract S generateState(); + + /** + * Called to produce data to the downstream subscribers. To emit data to a downstream subscriber + * call {@code observer.onNext(t)}. To signal an error condition call + * {@code observer.onError(throwable)} or throw an Exception. To signal the end of a data stream + * call {@code observer.onCompleted()}. Implementations of this method must follow the following + * rules. + * + *
    + *
  • Must not call {@code observer.onNext(t)} more than 1 time per invocation.
  • + *
  • Must not call {@code observer.onNext(t)} concurrently.
  • + *
+ * + * The value returned from an invocation of this method will be passed in as the {@code state} + * argument of the next invocation of this method. + * + * @param state + * the state value (from {@link #generateState()} on the first invocation or the + * previous invocation of this method. + * @param requested + * the amount of data requested. An observable emitted to the observer should not + * exceed this amount. + * @param observer + * the observer of data emitted by + * @return the next iteration's state value + */ + protected abstract S next(S state, long requested, Observer> observer); + + /** + * Clean up behavior that is executed after the downstream subscriber's subscription is + * unsubscribed. This method will be invoked exactly once. + * + * @param state + * the last state value returned from {@code next(S, Long, Observer)} or + * {@code generateState()} at the time when a terminal event is emitted from + * {@link #next(Object, long, Observer)} or unsubscribing. + */ + protected void onUnsubscribe(S state) { + + } + + /** + * Generates a synchronous {@link AsyncOnSubscribe} that calls the provided {@code next} + * function to generate data to downstream subscribers. + * + * @param generator + * generates the initial state value (see {@link #generateState()}) + * @param next + * produces data to the downstream subscriber (see + * {@link #next(Object, long, Observer) next(S, long, Observer)}) + * @return an OnSubscribe that emits data in a protocol compatible with back-pressure. + */ + @Experimental + public static OnSubscribe createSingleState(Func0 generator, + final Action3>> next) { + Func3>, S> nextFunc = + new Func3>, S>() { + @Override + public S call(S state, Long requested, Observer> subscriber) { + next.call(state, requested, subscriber); + return state; + }}; + return new AsyncOnSubscribeImpl(generator, nextFunc); + } + + /** + * Generates a synchronous {@link AsyncOnSubscribe} that calls the provided {@code next} + * function to generate data to downstream subscribers. + * + * This overload creates a AsyncOnSubscribe without an explicit clean up step. + * + * @param generator + * generates the initial state value (see {@link #generateState()}) + * @param next + * produces data to the downstream subscriber (see + * {@link #next(Object, long, Observer) next(S, long, Observer)}) + * @param onUnsubscribe + * clean up behavior (see {@link #onUnsubscribe(Object) onUnsubscribe(S)}) + * @return an OnSubscribe that emits data downstream in a protocol compatible with + * back-pressure. + */ + @Experimental + public static OnSubscribe createSingleState(Func0 generator, + final Action3>> next, + final Action1 onUnsubscribe) { + Func3>, S> nextFunc = + new Func3>, S>() { + @Override + public S call(S state, Long requested, Observer> subscriber) { + next.call(state, requested, subscriber); + return state; + }}; + return new AsyncOnSubscribeImpl(generator, nextFunc, onUnsubscribe); + } + + /** + * Generates a synchronous {@link AsyncOnSubscribe} that calls the provided {@code next} + * function to generate data to downstream subscribers. + * + * @param generator + * generates the initial state value (see {@link #generateState()}) + * @param next + * produces data to the downstream subscriber (see + * {@link #next(Object, long, Observer) next(S, long, Observer)}) + * @param onUnsubscribe + * clean up behavior (see {@link #onUnsubscribe(Object) onUnsubscribe(S)}) + * @return an OnSubscribe that emits data downstream in a protocol compatible with + * back-pressure. + */ + @Experimental + public static OnSubscribe createStateful(Func0 generator, + Func3>, ? extends S> next, + Action1 onUnsubscribe) { + return new AsyncOnSubscribeImpl(generator, next, onUnsubscribe); + } + + /** + * Generates a synchronous {@link AsyncOnSubscribe} that calls the provided {@code next} + * function to generate data to downstream subscribers. + * + * @param generator + * generates the initial state value (see {@link #generateState()}) + * @param next + * produces data to the downstream subscriber (see + * {@link #next(Object, long, Observer) next(S, long, Observer)}) + * @return an OnSubscribe that emits data downstream in a protocol compatible with + * back-pressure. + */ + @Experimental + public static OnSubscribe createStateful(Func0 generator, + Func3>, ? extends S> next) { + return new AsyncOnSubscribeImpl(generator, next); + } + + /** + * Generates a synchronous {@link AsyncOnSubscribe} that calls the provided {@code next} + * function to generate data to downstream subscribers. + * + * This overload creates a "state-less" AsyncOnSubscribe which does not have an explicit state + * value. This should be used when the {@code next} function closes over it's state. + * + * @param next + * produces data to the downstream subscriber (see + * {@link #next(Object, long, Observer) next(S, long, Observer)}) + * @return an OnSubscribe that emits data downstream in a protocol compatible with + * back-pressure. + */ + @Experimental + public static OnSubscribe createStateless(final Action2>> next) { + Func3>, Void> nextFunc = + new Func3>, Void>() { + @Override + public Void call(Void state, Long requested, Observer> subscriber) { + next.call(requested, subscriber); + return state; + }}; + return new AsyncOnSubscribeImpl(nextFunc); + } + + /** + * Generates a synchronous {@link AsyncOnSubscribe} that calls the provided {@code next} + * function to generate data to downstream subscribers. + * + * This overload creates a "state-less" AsyncOnSubscribe which does not have an explicit state + * value. This should be used when the {@code next} function closes over it's state. + * + * @param next + * produces data to the downstream subscriber (see + * {@link #next(Object, long, Observer) next(S, long, Observer)}) + * @param onUnsubscribe + * clean up behavior (see {@link #onUnsubscribe(Object) onUnsubscribe(S)}) + * @return an OnSubscribe that emits data downstream in a protocol compatible with + * back-pressure. + */ + @Experimental + public static OnSubscribe createStateless(final Action2>> next, + final Action0 onUnsubscribe) { + Func3>, Void> nextFunc = + new Func3>, Void>() { + @Override + public Void call(Void state, Long requested, Observer> subscriber) { + next.call(requested, subscriber); + return null; + }}; + Action1 wrappedOnUnsubscribe = new Action1() { + @Override + public void call(Void t) { + onUnsubscribe.call(); + }}; + return new AsyncOnSubscribeImpl(nextFunc, wrappedOnUnsubscribe); + } + + /** + * An implementation of AsyncOnSubscribe that delegates + * {@link AsyncOnSubscribe#next(Object, long, Observer)}, + * {@link AsyncOnSubscribe#generateState()}, and {@link AsyncOnSubscribe#onUnsubscribe(Object)} + * to provided functions/closures. + * + * @param + * the type of the user-defined state + * @param + * the type of compatible Subscribers + */ + private static final class AsyncOnSubscribeImpl extends AsyncOnSubscribe { + private final Func0 generator; + private final Func3>, ? extends S> next; + private final Action1 onUnsubscribe; + + private AsyncOnSubscribeImpl(Func0 generator, Func3>, ? extends S> next, Action1 onUnsubscribe) { + this.generator = generator; + this.next = next; + this.onUnsubscribe = onUnsubscribe; + } + + public AsyncOnSubscribeImpl(Func0 generator, Func3>, ? extends S> next) { + this(generator, next, null); + } + + public AsyncOnSubscribeImpl(Func3>, S> next, Action1 onUnsubscribe) { + this(null, next, onUnsubscribe); + } + + public AsyncOnSubscribeImpl(Func3>, S> nextFunc) { + this(null, nextFunc, null); + } + + @Override + protected S generateState() { + return generator == null ? null : generator.call(); + } + + @Override + protected S next(S state, long requested, Observer> observer) { + return next.call(state, requested, observer); + } + + @Override + protected void onUnsubscribe(S state) { + if (onUnsubscribe != null) + onUnsubscribe.call(state); + } + } + + @Override + public final void call(Subscriber actualSubscriber) { + S state = generateState(); + UnicastSubject> subject = UnicastSubject.> create(); + AsyncOuterSubscriber outerSubscriberProducer = new AsyncOuterSubscriber(this, state, subject); + actualSubscriber.add(outerSubscriberProducer); + Observable.concat(subject).unsafeSubscribe(Subscribers.wrap(actualSubscriber)); + actualSubscriber.setProducer(outerSubscriberProducer); + } + + private static class AsyncOuterSubscriber extends ConcurrentLinkedQueueimplements Producer, Subscription, Observer> { + /** */ + private static final long serialVersionUID = -7884904861928856832L; + + private volatile int isUnsubscribed; + @SuppressWarnings("rawtypes") + private static final AtomicIntegerFieldUpdater IS_UNSUBSCRIBED = AtomicIntegerFieldUpdater.newUpdater(AsyncOuterSubscriber.class, "isUnsubscribed"); + + private final AsyncOnSubscribe parent; + private final SerializedObserver> serializedSubscriber; + private final Set subscriptions = new HashSet(); + + private boolean hasTerminated = false; + private boolean onNextCalled = false; + + private S state; + + private final UnicastSubject> merger; + + public AsyncOuterSubscriber(AsyncOnSubscribe parent, S initialState, UnicastSubject> merger) { + this.parent = parent; + this.serializedSubscriber = new SerializedObserver>(this); + this.state = initialState; + this.merger = merger; + } + + @Override + public void unsubscribe() { + if (IS_UNSUBSCRIBED.compareAndSet(this, 0, 1)) { + // it's safe to process terminal behavior + if (isEmpty()) { + parent.onUnsubscribe(state); + } + for (Subscription s : subscriptions) { + if (!s.isUnsubscribed()) { + s.unsubscribe(); + } + } + } + } + + @Override + public boolean isUnsubscribed() { + return isUnsubscribed != 0; + } + + public void nextIteration(long requestCount) { + state = parent.next(state, requestCount, serializedSubscriber); + } + + @Override + public void request(long n) { + int size = 0; + Long r; + synchronized (this) { + size = size(); + add(n); + r = n; + } + if (size == 0) { + do { + // check if unsubscribed before doing any work + if (isUnsubscribed()) { + unsubscribe(); + return; + } + // otherwise try one iteration for a request of `numRequested` elements + try { + onNextCalled = false; + nextIteration(r); + if (onNextCalled) + r = poll(); + if (hasTerminated || isUnsubscribed()) { + parent.onUnsubscribe(state); + } + } catch (Throwable ex) { + handleThrownError(parent, state, ex); + return; + } + } while (r != null && !hasTerminated); + } + } + + private void handleThrownError(final AsyncOnSubscribe p, S st, Throwable ex) { + if (hasTerminated) { + RxJavaPlugins.getInstance().getErrorHandler().handleError(ex); + } else { + hasTerminated = true; + merger.onError(ex); + unsubscribe(); + } + } + + @Override + public void onCompleted() { + if (hasTerminated) { + throw new IllegalStateException("Terminal event already emitted."); + } + hasTerminated = true; + merger.onCompleted(); + } + + @Override + public void onError(Throwable e) { + if (hasTerminated) { + throw new IllegalStateException("Terminal event already emitted."); + } + hasTerminated = true; + merger.onError(e); + } + + // This exists simply to check if the subscription has already been + // terminated before getting access to the subscription + private static Subscription SUBSCRIPTION_SENTINEL = new BooleanSubscription(); + + @Override + public void onNext(final Observable t) { + if (onNextCalled) { + throw new IllegalStateException("onNext called multiple times!"); + } + onNextCalled = true; + if (hasTerminated) + return; + subscribeBufferToObservable(t); + } + + private void subscribeBufferToObservable(final Observable t) { + BufferUntilSubscriber buffer = BufferUntilSubscriber. create(); + final AtomicReference holder = new AtomicReference(null); + Subscription innerSubscription = t + .doOnTerminate(new Action0() { + @Override + public void call() { + if (!holder.compareAndSet(null, SUBSCRIPTION_SENTINEL)) { + Subscription h = holder.get(); + subscriptions.remove(h); + } + }}) + .subscribe(buffer); + + if (holder.compareAndSet(null, innerSubscription)) { + subscriptions.add(innerSubscription); + } + merger.onNext(buffer); + } + } + + private static final class UnicastSubject extends Observableimplements Observer { + public static UnicastSubject create() { + return new UnicastSubject(new State()); + } + + private State state; + + protected UnicastSubject(final State state) { + super(new OnSubscribe() { + @Override + public void call(Subscriber s) { + if (state.subscriber != null) { + s.onError(new IllegalStateException("There can be only one subscriber")); + } else { + state.subscriber = s; + } + } + }); + this.state = state; + } + + @Override + public void onCompleted() { + state.subscriber.onCompleted(); + } + + @Override + public void onError(Throwable e) { + state.subscriber.onError(e); + } + + @Override + public void onNext(T t) { + state.subscriber.onNext(t); + } + + private static class State { + private Subscriber subscriber; + } + } +} diff --git a/src/test/java/rx/observables/AsyncOnSubscribeTest.java b/src/test/java/rx/observables/AsyncOnSubscribeTest.java new file mode 100644 index 0000000000..92537dc455 --- /dev/null +++ b/src/test/java/rx/observables/AsyncOnSubscribeTest.java @@ -0,0 +1,408 @@ +package rx.observables; + +import static org.junit.Assert.assertEquals; + +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; + +import rx.Observable; +import rx.Observable.OnSubscribe; +import rx.Observer; +import rx.Subscription; +import rx.exceptions.TestException; +import rx.functions.Action0; +import rx.functions.Action1; +import rx.functions.Action2; +import rx.functions.Func0; +import rx.functions.Func3; +import rx.internal.util.RxRingBuffer; +import rx.observers.TestSubscriber; +import rx.schedulers.TestScheduler; + +@RunWith(MockitoJUnitRunner.class) +public class AsyncOnSubscribeTest { + + @Mock + public Observer o; + + TestSubscriber subscriber; + + @Before + public void setup() { + subscriber = new TestSubscriber(o); + } + + @Test + public void testSerializesConcurrentObservables() throws InterruptedException { + final TestScheduler scheduler = new TestScheduler(); + OnSubscribe os = AsyncOnSubscribe.createStateful(new Func0(){ + @Override + public Integer call() { + return 1; + }}, + new Func3>, Integer>(){ + @Override + public Integer call(Integer state, Long requested, Observer> observer) { + if (state == 1) { + Observable o1 = Observable.just(1, 2, 3, 4) + .delay(100, TimeUnit.MILLISECONDS, scheduler); + observer.onNext(o1); + } + else if (state == 2) { + Observable o = Observable.just(5, 6, 7, 8); + observer.onNext(o); + } + else + observer.onCompleted(); + return state + 1; + }}); + // initial request emits [[1, 2, 3, 4]] on delay + Observable.create(os).subscribe(subscriber); + // next request emits [[5, 6, 7, 8]] firing immediately + subscriber.requestMore(2); + // triggers delayed observable + scheduler.advanceTimeBy(100, TimeUnit.MILLISECONDS); + // final request completes + subscriber.requestMore(3); + subscriber.awaitTerminalEventAndUnsubscribeOnTimeout(100, TimeUnit.MILLISECONDS); + subscriber.assertNoErrors(); + subscriber.assertReceivedOnNext(Arrays.asList(new Integer[] {1, 2, 3, 4, 5, 6, 7, 8})); + subscriber.assertCompleted(); + } + + @Test + public void testSubscribedByBufferingOperator() { + final TestScheduler scheduler = new TestScheduler(); + OnSubscribe os = AsyncOnSubscribe.createStateless( + new Action2>>(){ + @Override + public void call(Long requested, Observer> observer) { + observer.onNext(Observable.range(1, requested.intValue())); + }}); + Observable.create(os).observeOn(scheduler).subscribe(subscriber); + scheduler.advanceTimeBy(10, TimeUnit.DAYS); + subscriber.assertNoErrors(); + subscriber.assertValueCount(RxRingBuffer.SIZE); + subscriber.assertNotCompleted(); + } + + @Test + public void testOnUnsubscribeHasCorrectState() throws InterruptedException { + final AtomicInteger lastState = new AtomicInteger(-1); + OnSubscribe os = AsyncOnSubscribe.createStateful(new Func0(){ + @Override + public Integer call() { + return 1; + }}, + new Func3>, Integer>(){ + @Override + public Integer call(Integer state, Long requested, Observer> observer) { + if (state < 3) { + observer.onNext(Observable.just(state)); + } + else + observer.onCompleted(); + return state + 1; + }}, + new Action1() { + @Override + public void call(Integer t) { + lastState.set(t); + }}); + Observable.create(os).subscribe(subscriber); // [[1]], state = 1 + subscriber.requestMore(2); // [[1]], state = 2 + subscriber.requestMore(3); // onComplete, state = 3 + subscriber.assertNoErrors(); + subscriber.assertReceivedOnNext(Arrays.asList(new Integer[] {1, 2})); + subscriber.assertCompleted(); + assertEquals("Final state when unsubscribing is not correct", 4, lastState.get()); + } + + @Test + public void testOnCompleteOuter() throws InterruptedException { + OnSubscribe os = AsyncOnSubscribe.createStateless(new Action2>>(){ + @Override + public void call(Long requested, Observer> observer) { + observer.onCompleted(); + }}); + Observable.create(os).subscribe(subscriber); + subscriber.assertNoErrors(); + subscriber.assertCompleted(); + subscriber.assertNoValues(); + } + + @Test + public void testTryOnNextTwice() throws InterruptedException { + OnSubscribe os = AsyncOnSubscribe.createStateless(new Action2>>(){ + @Override + public void call(Long requested, Observer> observer) { + observer.onNext(Observable.just(1)); + observer.onNext(Observable.just(2)); + } + }); + Observable.create(os).subscribe(subscriber); + subscriber.assertError(IllegalStateException.class); + subscriber.assertNotCompleted(); + subscriber.assertReceivedOnNext(Arrays.asList(new Integer[] {1})); + } + + @Test + public void testThrowException() throws InterruptedException { + OnSubscribe os = AsyncOnSubscribe.createStateless( + new Action2>>(){ + @Override + public void call(Long requested, Observer> observer) { + throw new TestException(); + }}); + Observable.create(os).subscribe(subscriber); + subscriber.assertError(TestException.class); + subscriber.assertNotCompleted(); + subscriber.assertNoValues(); + } + + @Test + public void testThrowExceptionAfterTerminal() throws InterruptedException { + OnSubscribe os = AsyncOnSubscribe.createStateful(new Func0(){ + @Override + public Integer call() { + return 1; + }}, + new Func3>, Integer>(){ + @Override + public Integer call(Integer state, Long requested, Observer> observer) { + observer.onCompleted(); + throw new TestException(); + }}); + Observable.create(os).subscribe(subscriber); + subscriber.assertNoErrors(); + subscriber.assertCompleted(); + subscriber.assertNoValues(); + } + + @Test + public void testOnNextAfterCompleted() throws InterruptedException { + OnSubscribe os = AsyncOnSubscribe.createStateful(new Func0(){ + @Override + public Integer call() { + return 1; + }}, + new Func3>, Integer>(){ + @Override + public Integer call(Integer state, Long requested, Observer> observer) { + observer.onCompleted(); + observer.onNext(Observable.just(1)); + return 1; + }}); + Observable.create(os).subscribe(subscriber); + subscriber.assertNoErrors(); + subscriber.assertCompleted(); + subscriber.assertNoValues(); + } + + @Test + public void testOnNextAfterError() throws InterruptedException { + OnSubscribe os = AsyncOnSubscribe.createStateful(new Func0(){ + @Override + public Integer call() { + return 1; + }}, + new Func3>, Integer>(){ + @Override + public Integer call(Integer state, Long requested, Observer> observer) { + observer.onError(new TestException()); + observer.onNext(Observable.just(1)); + return 1; + }}); + Observable.create(os).subscribe(subscriber); + subscriber.assertError(TestException.class); + subscriber.assertNotCompleted(); + subscriber.assertNoValues(); + } + + @Test + public void testEmittingEmptyObservable() throws InterruptedException { + OnSubscribe os = AsyncOnSubscribe.createStateful(new Func0(){ + @Override + public Integer call() { + return 1; + }}, + new Func3>, Integer>(){ + @Override + public Integer call(Integer state, Long requested, Observer> observer) { + observer.onNext(Observable.empty()); + observer.onCompleted(); + return state; + }}); + Observable.create(os).subscribe(subscriber); + subscriber.assertNoErrors(); + subscriber.assertCompleted(); + subscriber.assertNoValues(); + } + + @Test + public void testOnErrorOuter() throws InterruptedException { + OnSubscribe os = AsyncOnSubscribe.createStateful(new Func0(){ + @Override + public Integer call() { + return 1; + }}, + new Func3>, Integer>(){ + @Override + public Integer call(Integer state, Long requested, Observer> observer) { + observer.onError(new TestException()); + return state; + } + }); + Observable.create(os).subscribe(subscriber); + subscriber.assertError(TestException.class); + subscriber.assertNotCompleted(); + subscriber.assertNoValues(); + } + + @Test + public void testOnCompleteFollowedByOnErrorOuter() throws InterruptedException { + OnSubscribe os = AsyncOnSubscribe.createStateful(new Func0(){ + @Override + public Integer call() { + return 1; + }}, + new Func3>, Integer>(){ + @Override + public Integer call(Integer state, Long requested, Observer> observer) { + observer.onCompleted(); + observer.onError(new TestException()); + return state; + } + }); + Observable.create(os).subscribe(subscriber); + subscriber.assertCompleted(); + subscriber.assertNoErrors(); + subscriber.assertNoValues(); + } + + @Test + public void testUnsubscribesFromAllSelfTerminatedObservables() throws InterruptedException { + final AtomicInteger l1 = new AtomicInteger(); + final AtomicInteger l2 = new AtomicInteger(); + OnSubscribe os = AsyncOnSubscribe.createStateful(new Func0(){ + @Override + public Integer call() { + return 1; + }}, + new Func3>, Integer>(){ + @Override + public Integer call(Integer state, Long requested, Observer> observer) { + Observable o1; + switch (state) { + case 1: + o1 = Observable.just(1) + .doOnUnsubscribe(new Action0(){ + @Override + public void call() { + l1.incrementAndGet(); + }}); + break; + case 2: + o1 = Observable.just(2) + .doOnUnsubscribe(new Action0(){ + @Override + public void call() { + l2.incrementAndGet(); + }}); + break; + default: + observer.onCompleted(); + return null; + } + observer.onNext(o1); + return state + 1; + }}); + Observable.create(os).subscribe(subscriber); // [[1]] + subscriber.requestMore(2); // [[2]] + subscriber.requestMore(2); // onCompleted + subscriber.awaitTerminalEventAndUnsubscribeOnTimeout(100, TimeUnit.MILLISECONDS); + assertEquals("did not unsub from first observable after terminal", 1, l1.get()); + assertEquals("did not unsub from second observable after terminal", 1, l2.get()); + List onNextEvents = subscriber.getOnNextEvents(); + assertEquals(2, onNextEvents.size()); + assertEquals(new Integer(1), onNextEvents.get(0)); + assertEquals(new Integer(2), onNextEvents.get(1)); + subscriber.assertNoErrors(); + subscriber.assertCompleted(); + } + + @Test + public void testUnsubscribesFromAllNonTerminatedObservables() throws InterruptedException { + final AtomicInteger l1 = new AtomicInteger(); + final AtomicInteger l2 = new AtomicInteger(); + final TestScheduler scheduler = new TestScheduler(); + final AtomicReference sub = new AtomicReference(); + OnSubscribe os = AsyncOnSubscribe.createStateful(new Func0(){ + @Override + public Integer call() { + return 1; + }}, + new Func3>, Integer>(){ + @Override + public Integer call(Integer state, Long requested, Observer> observer) { + switch (state) { + case 1: + observer.onNext(Observable.just(1) + .subscribeOn(scheduler) + .doOnUnsubscribe(new Action0(){ + @Override + public void call() { + l1.incrementAndGet(); + }})); + break; + case 2: + observer.onNext(Observable.never() + .subscribeOn(scheduler) + .doOnUnsubscribe(new Action0(){ + @Override + public void call() { + l2.incrementAndGet(); + }})); + break; + case 3: + sub.get().unsubscribe(); + } + return state + 1; + }}); + Subscription subscription = Observable.create(os) + .observeOn(scheduler) + .subscribe(subscriber); + sub.set(subscription); + subscriber.assertNoValues(); + scheduler.triggerActions(); + subscriber.assertValue(1); + subscriber.assertNotCompleted(); + subscriber.assertNoErrors(); + assertEquals("did not unsub from 1st observable after terminal", 1, l1.get()); + assertEquals("did not unsub from Observable.never() inner obs", 1, l2.get()); + } + + private static class Foo {} + private static class Bar extends Foo {} + + @Test + public void testGenerics() { + AsyncOnSubscribe.createStateless(new Action2>>(){ + @Override + public void call(Long state, Observer> observer) { + if (state == null) + observer.onNext(Observable.just(new Foo())); + else + observer.onNext(Observable.just(new Bar())); + }}); + } +} From df2d934c8d0ef6e34e345c6ba130aaee15243aec Mon Sep 17 00:00:00 2001 From: Kevin Coughlin Date: Sun, 20 Sep 2015 23:20:03 -0400 Subject: [PATCH 56/81] Lint fixes for unnecessary unboxing. --- src/main/java/rx/internal/operators/OnSubscribeRedo.java | 2 +- src/main/java/rx/internal/operators/OperatorReplay.java | 2 +- src/main/java/rx/schedulers/TestScheduler.java | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/main/java/rx/internal/operators/OnSubscribeRedo.java b/src/main/java/rx/internal/operators/OnSubscribeRedo.java index 1431d4581c..cf517cb90e 100644 --- a/src/main/java/rx/internal/operators/OnSubscribeRedo.java +++ b/src/main/java/rx/internal/operators/OnSubscribeRedo.java @@ -111,7 +111,7 @@ public Observable> call(Observable call(Notification n, Notification term) { final int value = n.getValue(); - if (predicate.call(value, term.getThrowable()).booleanValue()) + if (predicate.call(value, term.getThrowable())) return Notification.createOnNext(value + 1); else return (Notification) term; diff --git a/src/main/java/rx/internal/operators/OperatorReplay.java b/src/main/java/rx/internal/operators/OperatorReplay.java index 6b42f1fb51..93d78ee14b 100644 --- a/src/main/java/rx/internal/operators/OperatorReplay.java +++ b/src/main/java/rx/internal/operators/OperatorReplay.java @@ -805,7 +805,7 @@ public void replay(InnerProducer output) { int sourceIndex = size; Integer destIndexObject = output.index(); - int destIndex = destIndexObject != null ? destIndexObject.intValue() : 0; + int destIndex = destIndexObject != null ? destIndexObject : 0; long r = output.get(); long r0 = r; diff --git a/src/main/java/rx/schedulers/TestScheduler.java b/src/main/java/rx/schedulers/TestScheduler.java index 358581ab74..fb7c0ef861 100644 --- a/src/main/java/rx/schedulers/TestScheduler.java +++ b/src/main/java/rx/schedulers/TestScheduler.java @@ -57,9 +57,9 @@ private static class CompareActionsByTime implements Comparator { @Override public int compare(TimedAction action1, TimedAction action2) { if (action1.time == action2.time) { - return Long.valueOf(action1.count).compareTo(Long.valueOf(action2.count)); + return Long.valueOf(action1.count).compareTo(action2.count); } else { - return Long.valueOf(action1.time).compareTo(Long.valueOf(action2.time)); + return Long.valueOf(action1.time).compareTo(action2.time); } } } From 135312ee15bc7e4e4a1d650aa13ebef3cdf5f309 Mon Sep 17 00:00:00 2001 From: Kevin Coughlin Date: Mon, 21 Sep 2015 19:02:23 -0400 Subject: [PATCH 57/81] Use ternary for comparison in place of Long.compareTo for Java 6 support. --- src/main/java/rx/schedulers/TestScheduler.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/rx/schedulers/TestScheduler.java b/src/main/java/rx/schedulers/TestScheduler.java index fb7c0ef861..c808a1a366 100644 --- a/src/main/java/rx/schedulers/TestScheduler.java +++ b/src/main/java/rx/schedulers/TestScheduler.java @@ -57,9 +57,9 @@ private static class CompareActionsByTime implements Comparator { @Override public int compare(TimedAction action1, TimedAction action2) { if (action1.time == action2.time) { - return Long.valueOf(action1.count).compareTo(action2.count); + return action1.count < action2.count ? -1 : ((action1.count > action2.count) ? 1 : 0); } else { - return Long.valueOf(action1.time).compareTo(action2.time); + return action1.time < action2.time ? -1 : ((action1.time > action2.time) ? 1 : 0); } } } From 9334b2a722ff780edeac32f0fd7ce4a854824f8a Mon Sep 17 00:00:00 2001 From: Artem Zinnatullin Date: Tue, 22 Sep 2015 01:39:46 +0300 Subject: [PATCH 58/81] Remove unused field updater from SubjectSubscriptionManager --- src/main/java/rx/subjects/SubjectSubscriptionManager.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/main/java/rx/subjects/SubjectSubscriptionManager.java b/src/main/java/rx/subjects/SubjectSubscriptionManager.java index a160c720a3..542d050c39 100644 --- a/src/main/java/rx/subjects/SubjectSubscriptionManager.java +++ b/src/main/java/rx/subjects/SubjectSubscriptionManager.java @@ -40,8 +40,6 @@ = AtomicReferenceFieldUpdater.newUpdater(SubjectSubscriptionManager.class, State.class, "state"); /** Stores the latest value or the terminal value for some Subjects. */ volatile Object latest; - static final AtomicReferenceFieldUpdater LATEST_UPDATER - = AtomicReferenceFieldUpdater.newUpdater(SubjectSubscriptionManager.class, Object.class, "latest"); /** Indicates that the subject is active (cheaper than checking the state).*/ boolean active = true; /** Action called when a new subscriber subscribes but before it is added to the state. */ From f9ab970569e8bbdd36fba26a129815d9ce0a2040 Mon Sep 17 00:00:00 2001 From: Steve Gury Date: Mon, 28 Sep 2015 14:27:48 -0700 Subject: [PATCH 59/81] Fix typo in a comment inside Observable.subscribe sigificent -> significant alreay -> already --- src/main/java/rx/Observable.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/main/java/rx/Observable.java b/src/main/java/rx/Observable.java index 5d6d77c15f..0b50ef9268 100644 --- a/src/main/java/rx/Observable.java +++ b/src/main/java/rx/Observable.java @@ -7821,7 +7821,8 @@ private static Subscription subscribe(Subscriber subscriber, Obse subscriber = new SafeSubscriber(subscriber); } - // The code below is exactly the same an unsafeSubscribe but not used because it would add a sigificent depth to alreay huge call stacks. + // The code below is exactly the same an unsafeSubscribe but not used because it would + // add a significant depth to already huge call stacks. try { // allow the hook to intercept and/or decorate hook.onSubscribeStart(observable, observable.onSubscribe).call(subscriber); From d6aabbac9db5c591831a4443d9a21a2861ccd9ca Mon Sep 17 00:00:00 2001 From: Artem Zinnatullin Date: Sun, 20 Sep 2015 04:23:37 +0300 Subject: [PATCH 60/81] Make field final and remove unnecessary unboxing in OnSubscribeRedo.RetryWithPredicate --- src/main/java/rx/internal/operators/OnSubscribeRedo.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/rx/internal/operators/OnSubscribeRedo.java b/src/main/java/rx/internal/operators/OnSubscribeRedo.java index cf517cb90e..48521d00b1 100644 --- a/src/main/java/rx/internal/operators/OnSubscribeRedo.java +++ b/src/main/java/rx/internal/operators/OnSubscribeRedo.java @@ -98,7 +98,7 @@ public Notification call(Notification terminalNotification) { } public static final class RetryWithPredicate implements Func1>, Observable>> { - private Func2 predicate; + private final Func2 predicate; public RetryWithPredicate(Func2 predicate) { this.predicate = predicate; From 8859bcfcce6872b14dcd69005c835c26a70fd312 Mon Sep 17 00:00:00 2001 From: Artem Zinnatullin Date: Sun, 20 Sep 2015 03:25:11 +0300 Subject: [PATCH 61/81] Fix synchronization on non-final field in BufferUntilSubscriber --- src/main/java/rx/internal/operators/BufferUntilSubscriber.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/rx/internal/operators/BufferUntilSubscriber.java b/src/main/java/rx/internal/operators/BufferUntilSubscriber.java index 737b8d2bee..e4722c9a60 100644 --- a/src/main/java/rx/internal/operators/BufferUntilSubscriber.java +++ b/src/main/java/rx/internal/operators/BufferUntilSubscriber.java @@ -70,7 +70,7 @@ boolean casObserverRef(Observer expected, Observer next) return OBSERVER_UPDATER.compareAndSet(this, expected, next); } - Object guard = new Object(); + final Object guard = new Object(); /* protected by guard */ boolean emitting = false; From 3dd46e68937f3e9d0dca1bc2a709aac4607815d8 Mon Sep 17 00:00:00 2001 From: akarnokd Date: Sat, 19 Sep 2015 11:36:15 +0200 Subject: [PATCH 62/81] Fix to a bunch of bugs and issues with AsyncOnSubscribe --- .../java/rx/observables/AsyncOnSubscribe.java | 335 +++++++++++++----- .../rx/observables/AsyncOnSubscribeTest.java | 76 +++- 2 files changed, 312 insertions(+), 99 deletions(-) diff --git a/src/main/java/rx/observables/AsyncOnSubscribe.java b/src/main/java/rx/observables/AsyncOnSubscribe.java index d4a12b0245..84cb4c98e4 100644 --- a/src/main/java/rx/observables/AsyncOnSubscribe.java +++ b/src/main/java/rx/observables/AsyncOnSubscribe.java @@ -16,30 +16,19 @@ package rx.observables; -import java.util.HashSet; -import java.util.Set; -import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.*; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; -import java.util.concurrent.atomic.AtomicReference; +import rx.*; import rx.Observable; import rx.Observable.OnSubscribe; import rx.Observer; -import rx.Producer; -import rx.Subscriber; -import rx.Subscription; import rx.annotations.Experimental; -import rx.functions.Action0; -import rx.functions.Action1; -import rx.functions.Action2; -import rx.functions.Action3; -import rx.functions.Func0; -import rx.functions.Func3; -import rx.internal.operators.BufferUntilSubscriber; -import rx.observers.SerializedObserver; -import rx.observers.Subscribers; +import rx.functions.*; +import rx.internal.operators.*; +import rx.observers.*; import rx.plugins.RxJavaPlugins; -import rx.subscriptions.BooleanSubscription; +import rx.subscriptions.CompositeSubscription; ; /** * A utility class to create {@code OnSubscribe} functions that respond correctly to back @@ -311,35 +300,77 @@ protected void onUnsubscribe(S state) { } @Override - public final void call(Subscriber actualSubscriber) { - S state = generateState(); + public final void call(final Subscriber actualSubscriber) { + S state; + try { + state = generateState(); + } catch (Throwable ex) { + actualSubscriber.onError(ex); + return; + } UnicastSubject> subject = UnicastSubject.> create(); - AsyncOuterSubscriber outerSubscriberProducer = new AsyncOuterSubscriber(this, state, subject); - actualSubscriber.add(outerSubscriberProducer); - Observable.concat(subject).unsafeSubscribe(Subscribers.wrap(actualSubscriber)); - actualSubscriber.setProducer(outerSubscriberProducer); + + final AsyncOuterManager outerProducer = new AsyncOuterManager(this, state, subject); + + Subscriber concatSubscriber = new Subscriber() { + @Override + public void onNext(T t) { + actualSubscriber.onNext(t); + } + + @Override + public void onError(Throwable e) { + actualSubscriber.onError(e); + } + + @Override + public void onCompleted() { + actualSubscriber.onCompleted(); + } + + @Override + public void setProducer(Producer p) { + outerProducer.setConcatProducer(p); + } + }; + + subject.onBackpressureBuffer().concatMap(new Func1, Observable>() { + @Override + public Observable call(Observable v) { + return v.onBackpressureBuffer(); + } + }).unsafeSubscribe(concatSubscriber); + + actualSubscriber.add(concatSubscriber); + actualSubscriber.add(outerProducer); + actualSubscriber.setProducer(outerProducer); + } - private static class AsyncOuterSubscriber extends ConcurrentLinkedQueueimplements Producer, Subscription, Observer> { - /** */ - private static final long serialVersionUID = -7884904861928856832L; + static final class AsyncOuterManager implements Producer, Subscription, Observer> { private volatile int isUnsubscribed; @SuppressWarnings("rawtypes") - private static final AtomicIntegerFieldUpdater IS_UNSUBSCRIBED = AtomicIntegerFieldUpdater.newUpdater(AsyncOuterSubscriber.class, "isUnsubscribed"); + private static final AtomicIntegerFieldUpdater IS_UNSUBSCRIBED = AtomicIntegerFieldUpdater.newUpdater(AsyncOuterManager.class, "isUnsubscribed"); private final AsyncOnSubscribe parent; private final SerializedObserver> serializedSubscriber; - private final Set subscriptions = new HashSet(); + private final CompositeSubscription subscriptions = new CompositeSubscription(); - private boolean hasTerminated = false; - private boolean onNextCalled = false; + private boolean hasTerminated; + private boolean onNextCalled; private S state; private final UnicastSubject> merger; - - public AsyncOuterSubscriber(AsyncOnSubscribe parent, S initialState, UnicastSubject> merger) { + + boolean emitting; + List requests; + Producer concatProducer; + + long expectedDelivery; + + public AsyncOuterManager(AsyncOnSubscribe parent, S initialState, UnicastSubject> merger) { this.parent = parent; this.serializedSubscriber = new SerializedObserver>(this); this.state = initialState; @@ -349,18 +380,25 @@ public AsyncOuterSubscriber(AsyncOnSubscribe parent, S initialState, Unica @Override public void unsubscribe() { if (IS_UNSUBSCRIBED.compareAndSet(this, 0, 1)) { - // it's safe to process terminal behavior - if (isEmpty()) { - parent.onUnsubscribe(state); - } - for (Subscription s : subscriptions) { - if (!s.isUnsubscribed()) { - s.unsubscribe(); + synchronized (this) { + if (emitting) { + requests = new ArrayList(); + requests.add(0L); + return; } + emitting = true; } + cleanup(); } } + void setConcatProducer(Producer p) { + if (concatProducer != null) { + throw new IllegalStateException("setConcatProducer may be called at most once!"); + } + concatProducer = p; + } + @Override public boolean isUnsubscribed() { return isUnsubscribed != 0; @@ -369,47 +407,149 @@ public boolean isUnsubscribed() { public void nextIteration(long requestCount) { state = parent.next(state, requestCount, serializedSubscriber); } + + void cleanup() { + subscriptions.unsubscribe(); + try { + parent.onUnsubscribe(state); + } catch (Throwable ex) { + handleThrownError(ex); + } + } @Override public void request(long n) { - int size = 0; - Long r; + if (n == 0) { + return; + } + if (n < 0) { + throw new IllegalStateException("Request can't be negative! " + n); + } + boolean quit = false; synchronized (this) { - size = size(); - add(n); - r = n; + if (emitting) { + List q = requests; + if (q == null) { + q = new ArrayList(); + requests = q; + } + q.add(n); + + quit = true; + } else { + emitting = true; + } + } + + concatProducer.request(n); + + if (quit) { + return; + } + + if (tryEmit(n)) { + return; } - if (size == 0) { - do { - // check if unsubscribed before doing any work - if (isUnsubscribed()) { - unsubscribe(); + for (;;) { + List q; + synchronized (this) { + q = requests; + if (q == null) { + emitting = false; return; } - // otherwise try one iteration for a request of `numRequested` elements - try { - onNextCalled = false; - nextIteration(r); - if (onNextCalled) - r = poll(); - if (hasTerminated || isUnsubscribed()) { - parent.onUnsubscribe(state); - } - } catch (Throwable ex) { - handleThrownError(parent, state, ex); + requests = null; + } + + for (long r : q) { + if (tryEmit(r)) { return; } - } while (r != null && !hasTerminated); + } } } - private void handleThrownError(final AsyncOnSubscribe p, S st, Throwable ex) { + /** + * Called when a source has produced less than its provision (completed prematurely); this will trigger the generation of another + * source that will hopefully emit the missing amount. + * @param n the missing amount to produce via a new source. + */ + public void requestRemaining(long n) { + if (n == 0) { + return; + } + if (n < 0) { + throw new IllegalStateException("Request can't be negative! " + n); + } + synchronized (this) { + if (emitting) { + List q = requests; + if (q == null) { + q = new ArrayList(); + requests = q; + } + q.add(n); + + return; + } + emitting = true; + } + + if (tryEmit(n)) { + return; + } + for (;;) { + List q; + synchronized (this) { + q = requests; + if (q == null) { + emitting = false; + return; + } + requests = null; + } + + for (long r : q) { + if (tryEmit(r)) { + return; + } + } + } + } + + boolean tryEmit(long n) { + if (isUnsubscribed()) { + cleanup(); + return true; + } + + try { + onNextCalled = false; + expectedDelivery = n; + nextIteration(n); + + if (hasTerminated || isUnsubscribed()) { + cleanup(); + return true; + } + if (!onNextCalled) { + handleThrownError(new IllegalStateException("No events emitted!")); + return true; + } + } catch (Throwable ex) { + handleThrownError(ex); + return true; + } + return false; + } + + private void handleThrownError(Throwable ex) { if (hasTerminated) { RxJavaPlugins.getInstance().getErrorHandler().handleError(ex); } else { hasTerminated = true; merger.onError(ex); - unsubscribe(); + cleanup(); } } @@ -431,10 +571,6 @@ public void onError(Throwable e) { merger.onError(e); } - // This exists simply to check if the subscription has already been - // terminated before getting access to the subscription - private static Subscription SUBSCRIPTION_SENTINEL = new BooleanSubscription(); - @Override public void onNext(final Observable t) { if (onNextCalled) { @@ -447,27 +583,43 @@ public void onNext(final Observable t) { } private void subscribeBufferToObservable(final Observable t) { - BufferUntilSubscriber buffer = BufferUntilSubscriber. create(); - final AtomicReference holder = new AtomicReference(null); - Subscription innerSubscription = t - .doOnTerminate(new Action0() { + final BufferUntilSubscriber buffer = BufferUntilSubscriber. create(); + + final long expected = expectedDelivery; + final Subscriber s = new Subscriber() { + long remaining = expected; + @Override + public void onNext(T t) { + remaining--; + buffer.onNext(t); + } + @Override + public void onError(Throwable e) { + buffer.onError(e); + } + @Override + public void onCompleted() { + buffer.onCompleted(); + long r = remaining; + if (r > 0) { + requestRemaining(r); + } + } + }; + subscriptions.add(s); + + t.doOnTerminate(new Action0() { @Override public void call() { - if (!holder.compareAndSet(null, SUBSCRIPTION_SENTINEL)) { - Subscription h = holder.get(); - subscriptions.remove(h); - } + subscriptions.remove(s); }}) - .subscribe(buffer); + .subscribe(s); - if (holder.compareAndSet(null, innerSubscription)) { - subscriptions.add(innerSubscription); - } merger.onNext(buffer); } } - private static final class UnicastSubject extends Observableimplements Observer { + static final class UnicastSubject extends Observableimplements Observer { public static UnicastSubject create() { return new UnicastSubject(new State()); } @@ -475,16 +627,7 @@ public static UnicastSubject create() { private State state; protected UnicastSubject(final State state) { - super(new OnSubscribe() { - @Override - public void call(Subscriber s) { - if (state.subscriber != null) { - s.onError(new IllegalStateException("There can be only one subscriber")); - } else { - state.subscriber = s; - } - } - }); + super(state); this.state = state; } @@ -503,8 +646,18 @@ public void onNext(T t) { state.subscriber.onNext(t); } - private static class State { + static final class State implements OnSubscribe { private Subscriber subscriber; + @Override + public void call(Subscriber s) { + synchronized (this) { + if (subscriber == null) { + subscriber = s; + return; + } + } + s.onError(new IllegalStateException("There can be only one subscriber")); + } } } } diff --git a/src/test/java/rx/observables/AsyncOnSubscribeTest.java b/src/test/java/rx/observables/AsyncOnSubscribeTest.java index 92537dc455..633d229921 100644 --- a/src/test/java/rx/observables/AsyncOnSubscribeTest.java +++ b/src/test/java/rx/observables/AsyncOnSubscribeTest.java @@ -38,7 +38,7 @@ public class AsyncOnSubscribeTest { @Before public void setup() { - subscriber = new TestSubscriber(o); + subscriber = new TestSubscriber(o, 0L); } @Test @@ -68,14 +68,20 @@ else if (state == 2) { // initial request emits [[1, 2, 3, 4]] on delay Observable.create(os).subscribe(subscriber); // next request emits [[5, 6, 7, 8]] firing immediately - subscriber.requestMore(2); + subscriber.requestMore(2); // triggers delayed observable scheduler.advanceTimeBy(100, TimeUnit.MILLISECONDS); + subscriber.assertNoErrors(); + subscriber.assertValues(1, 2); // final request completes subscriber.requestMore(3); - subscriber.awaitTerminalEventAndUnsubscribeOnTimeout(100, TimeUnit.MILLISECONDS); subscriber.assertNoErrors(); - subscriber.assertReceivedOnNext(Arrays.asList(new Integer[] {1, 2, 3, 4, 5, 6, 7, 8})); + subscriber.assertValues(1, 2, 3, 4, 5); + + subscriber.requestMore(3); + + subscriber.assertNoErrors(); + subscriber.assertValues(1, 2, 3, 4, 5, 6, 7, 8); subscriber.assertCompleted(); } @@ -89,6 +95,7 @@ public void call(Long requested, Observer> observe observer.onNext(Observable.range(1, requested.intValue())); }}); Observable.create(os).observeOn(scheduler).subscribe(subscriber); + subscriber.requestMore(RxRingBuffer.SIZE); scheduler.advanceTimeBy(10, TimeUnit.DAYS); subscriber.assertNoErrors(); subscriber.assertValueCount(RxRingBuffer.SIZE); @@ -118,7 +125,8 @@ public Integer call(Integer state, Long requested, Observer> observe observer.onCompleted(); }}); Observable.create(os).subscribe(subscriber); + subscriber.requestMore(1); subscriber.assertNoErrors(); subscriber.assertCompleted(); subscriber.assertNoValues(); @@ -150,6 +159,7 @@ public void call(Long requested, Observer> observe } }); Observable.create(os).subscribe(subscriber); + subscriber.requestMore(1); subscriber.assertError(IllegalStateException.class); subscriber.assertNotCompleted(); subscriber.assertReceivedOnNext(Arrays.asList(new Integer[] {1})); @@ -164,6 +174,7 @@ public void call(Long requested, Observer> observe throw new TestException(); }}); Observable.create(os).subscribe(subscriber); + subscriber.requestMore(1); subscriber.assertError(TestException.class); subscriber.assertNotCompleted(); subscriber.assertNoValues(); @@ -183,6 +194,7 @@ public Integer call(Integer state, Long requested, Observer> observer) { switch (state) { case 1: - observer.onNext(Observable.just(1) + observer.onNext(Observable.range(1, requested.intValue()) .subscribeOn(scheduler) .doOnUnsubscribe(new Action0(){ @Override @@ -383,8 +401,11 @@ public void call() { .subscribe(subscriber); sub.set(subscription); subscriber.assertNoValues(); + subscriber.requestMore(1); + scheduler.triggerActions(); + subscriber.requestMore(1); scheduler.triggerActions(); - subscriber.assertValue(1); + subscriber.assertValueCount(2); subscriber.assertNotCompleted(); subscriber.assertNoErrors(); assertEquals("did not unsub from 1st observable after terminal", 1, l1.get()); @@ -405,4 +426,43 @@ public void call(Long state, Observer> observer) { observer.onNext(Observable.just(new Bar())); }}); } + + @Test + public void testUnderdeliveryCorrection() { + OnSubscribe os = AsyncOnSubscribe.createStateful(new Func0(){ + @Override + public Integer call() { + return 1; + }}, + new Func3>, Integer>(){ + @Override + public Integer call(Integer state, Long requested, Observer> observer) { + switch (state) { + case 1: + observer.onNext(Observable.just(1)); + break; + default: + observer.onNext(Observable.range(1, requested.intValue())); + break; + } + return state + 1; + }}); + Observable.create(os).subscribe(subscriber); + + subscriber.assertNoErrors(); + subscriber.assertNotCompleted(); + subscriber.assertNoValues(); + + subscriber.requestMore(2); + + subscriber.assertNoErrors(); + subscriber.assertValueCount(2); + + subscriber.requestMore(5); + + subscriber.assertNoErrors(); + subscriber.assertValueCount(7); + + subscriber.assertNotCompleted(); + } } From 6f73a02f02353b48785bd78317308dafd445930f Mon Sep 17 00:00:00 2001 From: akarnokd Date: Tue, 29 Sep 2015 09:53:19 +0200 Subject: [PATCH 63/81] Fix for take() reentrancy bug. --- .../rx/internal/operators/OperatorTake.java | 4 ++-- .../internal/operators/OperatorTakeTest.java | 21 +++++++++++++++++++ 2 files changed, 23 insertions(+), 2 deletions(-) diff --git a/src/main/java/rx/internal/operators/OperatorTake.java b/src/main/java/rx/internal/operators/OperatorTake.java index 31811537b5..d1cc1cbd09 100644 --- a/src/main/java/rx/internal/operators/OperatorTake.java +++ b/src/main/java/rx/internal/operators/OperatorTake.java @@ -68,8 +68,8 @@ public void onError(Throwable e) { @Override public void onNext(T i) { - if (!isUnsubscribed()) { - boolean stop = ++count >= limit; + if (!isUnsubscribed() && count++ < limit) { + boolean stop = count == limit; child.onNext(i); if (stop && !completed) { completed = true; diff --git a/src/test/java/rx/internal/operators/OperatorTakeTest.java b/src/test/java/rx/internal/operators/OperatorTakeTest.java index 3384445d5b..4173f08892 100644 --- a/src/test/java/rx/internal/operators/OperatorTakeTest.java +++ b/src/test/java/rx/internal/operators/OperatorTakeTest.java @@ -32,6 +32,7 @@ import rx.functions.*; import rx.observers.*; import rx.schedulers.Schedulers; +import rx.subjects.PublishSubject; public class OperatorTakeTest { @@ -417,4 +418,24 @@ public void onNext(Integer t) { ts.assertError(TestException.class); ts.assertNotCompleted(); } + + @Test + public void testReentrantTake() { + final PublishSubject source = PublishSubject.create(); + + TestSubscriber ts = new TestSubscriber(); + + source.take(1).doOnNext(new Action1() { + @Override + public void call(Integer v) { + source.onNext(2); + } + }).subscribe(ts); + + source.onNext(1); + + ts.assertValue(1); + ts.assertNoErrors(); + ts.assertCompleted(); + } } From bf0f3e95a086f6965d010da2e4f863a7362f780b Mon Sep 17 00:00:00 2001 From: akarnokd Date: Thu, 13 Aug 2015 00:49:23 +0200 Subject: [PATCH 64/81] Schedulers shutdown capability. --- .../schedulers/EventLoopsScheduler.java | 65 +++++++-- .../GenericScheduledExecutorService.java | 65 +++++++-- .../internal/schedulers/NewThreadWorker.java | 2 + .../schedulers/SchedulerLifecycle.java | 20 +++ .../java/rx/internal/util/ObjectPool.java | 76 ++++++---- .../java/rx/internal/util/RxRingBuffer.java | 8 +- .../rx/schedulers/CachedThreadScheduler.java | 132 +++++++++++++----- .../java/rx/schedulers/ExecutorScheduler.java | 2 +- src/main/java/rx/schedulers/Schedulers.java | 53 ++++++- .../schedulers/SchedulerLifecycleTest.java | 127 +++++++++++++++++ 10 files changed, 456 insertions(+), 94 deletions(-) rename src/main/java/rx/{ => internal}/schedulers/GenericScheduledExecutorService.java (58%) create mode 100644 src/main/java/rx/internal/schedulers/SchedulerLifecycle.java create mode 100644 src/test/java/rx/internal/schedulers/SchedulerLifecycleTest.java diff --git a/src/main/java/rx/internal/schedulers/EventLoopsScheduler.java b/src/main/java/rx/internal/schedulers/EventLoopsScheduler.java index 986ea6d467..d901304680 100644 --- a/src/main/java/rx/internal/schedulers/EventLoopsScheduler.java +++ b/src/main/java/rx/internal/schedulers/EventLoopsScheduler.java @@ -16,13 +16,14 @@ package rx.internal.schedulers; import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicReference; import rx.*; import rx.functions.Action0; import rx.internal.util.*; import rx.subscriptions.*; -public class EventLoopsScheduler extends Scheduler { +public class EventLoopsScheduler extends Scheduler implements SchedulerLifecycle { /** Manages a fixed number of workers. */ private static final String THREAD_NAME_PREFIX = "RxComputationThreadPool-"; private static final RxThreadFactory THREAD_FACTORY = new RxThreadFactory(THREAD_NAME_PREFIX); @@ -44,40 +45,82 @@ public class EventLoopsScheduler extends Scheduler { } MAX_THREADS = max; } + + static final PoolWorker SHUTDOWN_WORKER; + static { + SHUTDOWN_WORKER = new PoolWorker(new RxThreadFactory("RxComputationShutdown-")); + SHUTDOWN_WORKER.unsubscribe(); + } + static final class FixedSchedulerPool { final int cores; final PoolWorker[] eventLoops; long n; - FixedSchedulerPool() { + FixedSchedulerPool(int maxThreads) { // initialize event loops - this.cores = MAX_THREADS; - this.eventLoops = new PoolWorker[cores]; - for (int i = 0; i < cores; i++) { + this.cores = maxThreads; + this.eventLoops = new PoolWorker[maxThreads]; + for (int i = 0; i < maxThreads; i++) { this.eventLoops[i] = new PoolWorker(THREAD_FACTORY); } } public PoolWorker getEventLoop() { + int c = cores; + if (c == 0) { + return SHUTDOWN_WORKER; + } // simple round robin, improvements to come - return eventLoops[(int)(n++ % cores)]; + return eventLoops[(int)(n++ % c)]; + } + + public void shutdown() { + for (PoolWorker w : eventLoops) { + w.unsubscribe(); + } } } + /** This will indicate no pool is active. */ + static final FixedSchedulerPool NONE = new FixedSchedulerPool(0); - final FixedSchedulerPool pool; + final AtomicReference pool; /** * Create a scheduler with pool size equal to the available processor * count and using least-recent worker selection policy. */ public EventLoopsScheduler() { - pool = new FixedSchedulerPool(); + this.pool = new AtomicReference(NONE); + start(); } @Override public Worker createWorker() { - return new EventLoopWorker(pool.getEventLoop()); + return new EventLoopWorker(pool.get().getEventLoop()); + } + + @Override + public void start() { + FixedSchedulerPool update = new FixedSchedulerPool(MAX_THREADS); + if (!pool.compareAndSet(NONE, update)) { + update.shutdown(); + } + } + + @Override + public void shutdown() { + for (;;) { + FixedSchedulerPool curr = pool.get(); + if (curr == NONE) { + return; + } + if (pool.compareAndSet(curr, NONE)) { + curr.shutdown(); + return; + } + } } /** @@ -87,7 +130,7 @@ public Worker createWorker() { * @return the subscription */ public Subscription scheduleDirect(Action0 action) { - PoolWorker pw = pool.getEventLoop(); + PoolWorker pw = pool.get().getEventLoop(); return pw.scheduleActual(action, -1, TimeUnit.NANOSECONDS); } @@ -137,4 +180,4 @@ private static final class PoolWorker extends NewThreadWorker { super(threadFactory); } } -} +} \ No newline at end of file diff --git a/src/main/java/rx/schedulers/GenericScheduledExecutorService.java b/src/main/java/rx/internal/schedulers/GenericScheduledExecutorService.java similarity index 58% rename from src/main/java/rx/schedulers/GenericScheduledExecutorService.java rename to src/main/java/rx/internal/schedulers/GenericScheduledExecutorService.java index ca133275e7..e4c3e9ba61 100644 --- a/src/main/java/rx/schedulers/GenericScheduledExecutorService.java +++ b/src/main/java/rx/internal/schedulers/GenericScheduledExecutorService.java @@ -13,13 +13,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package rx.schedulers; +package rx.internal.schedulers; + +import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicReference; import rx.Scheduler; -import rx.internal.schedulers.NewThreadWorker; import rx.internal.util.RxThreadFactory; - -import java.util.concurrent.*; +import rx.schedulers.*; /** * A default {@link ScheduledExecutorService} that can be used for scheduling actions when a {@link Scheduler} implementation doesn't have that ability. @@ -30,15 +31,29 @@ * the work asynchronously on the appropriate {@link Scheduler} implementation. This means for example that you would not use this approach * along with {@link TrampolineScheduler} or {@link ImmediateScheduler}. */ -/* package */final class GenericScheduledExecutorService { +public final class GenericScheduledExecutorService implements SchedulerLifecycle{ private static final String THREAD_NAME_PREFIX = "RxScheduledExecutorPool-"; private static final RxThreadFactory THREAD_FACTORY = new RxThreadFactory(THREAD_NAME_PREFIX); - private final static GenericScheduledExecutorService INSTANCE = new GenericScheduledExecutorService(); - private final ScheduledExecutorService executor; - + /* Schedulers needs acces to this in order to work with the lifecycle. */ + public final static GenericScheduledExecutorService INSTANCE = new GenericScheduledExecutorService(); + + private final AtomicReference executor; + + static final ScheduledExecutorService NONE; + static { + NONE = Executors.newScheduledThreadPool(0); + NONE.shutdownNow(); + } + private GenericScheduledExecutorService() { + executor = new AtomicReference(NONE); + start(); + } + + @Override + public void start() { int count = Runtime.getRuntime().availableProcessors(); if (count > 4) { count = count / 2; @@ -47,21 +62,41 @@ private GenericScheduledExecutorService() { if (count > 8) { count = 8; } + ScheduledExecutorService exec = Executors.newScheduledThreadPool(count, THREAD_FACTORY); - if (!NewThreadWorker.tryEnableCancelPolicy(exec)) { - if (exec instanceof ScheduledThreadPoolExecutor) { - NewThreadWorker.registerExecutor((ScheduledThreadPoolExecutor)exec); + if (executor.compareAndSet(NONE, exec)) { + if (!NewThreadWorker.tryEnableCancelPolicy(exec)) { + if (exec instanceof ScheduledThreadPoolExecutor) { + NewThreadWorker.registerExecutor((ScheduledThreadPoolExecutor)exec); + } } + return; + } else { + exec.shutdownNow(); } - executor = exec; } - + + @Override + public void shutdown() { + for (;;) { + ScheduledExecutorService exec = executor.get(); + if (exec == NONE) { + return; + } + if (executor.compareAndSet(exec, NONE)) { + NewThreadWorker.deregisterExecutor(exec); + exec.shutdownNow(); + return; + } + } + } + /** * See class Javadoc for information on what this is for and how to use. * * @return {@link ScheduledExecutorService} for generic use. */ public static ScheduledExecutorService getInstance() { - return INSTANCE.executor; + return INSTANCE.executor.get(); } -} +} \ No newline at end of file diff --git a/src/main/java/rx/internal/schedulers/NewThreadWorker.java b/src/main/java/rx/internal/schedulers/NewThreadWorker.java index 4c47936871..0103a609ff 100644 --- a/src/main/java/rx/internal/schedulers/NewThreadWorker.java +++ b/src/main/java/rx/internal/schedulers/NewThreadWorker.java @@ -82,6 +82,8 @@ public void run() { }, PURGE_FREQUENCY, PURGE_FREQUENCY, TimeUnit.MILLISECONDS); break; + } else { + exec.shutdownNow(); } } while (true); diff --git a/src/main/java/rx/internal/schedulers/SchedulerLifecycle.java b/src/main/java/rx/internal/schedulers/SchedulerLifecycle.java new file mode 100644 index 0000000000..a9c7bd3f12 --- /dev/null +++ b/src/main/java/rx/internal/schedulers/SchedulerLifecycle.java @@ -0,0 +1,20 @@ +package rx.internal.schedulers; + +/** + * Represents the capability of a Scheduler to be start or shut down its maintained + * threads. + */ +public interface SchedulerLifecycle { + /** + * Allows the Scheduler instance to start threads + * and accept tasks on them. + *

Implementations should make sure the call is idempotent and threadsafe. + */ + void start(); + /** + * Instructs the Scheduler instance to stop threads + * and stop accepting tasks on any outstanding Workers. + *

Implementations should make sure the call is idempotent and threadsafe. + */ + void shutdown(); +} \ No newline at end of file diff --git a/src/main/java/rx/internal/util/ObjectPool.java b/src/main/java/rx/internal/util/ObjectPool.java index 8a059068a8..504c10cad4 100644 --- a/src/main/java/rx/internal/util/ObjectPool.java +++ b/src/main/java/rx/internal/util/ObjectPool.java @@ -18,20 +18,22 @@ package rx.internal.util; import java.util.Queue; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.TimeUnit; +import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicReference; -import rx.Scheduler; +import rx.Scheduler.Worker; import rx.functions.Action0; -import rx.internal.util.unsafe.MpmcArrayQueue; -import rx.internal.util.unsafe.UnsafeAccess; +import rx.internal.schedulers.SchedulerLifecycle; +import rx.internal.util.unsafe.*; import rx.schedulers.Schedulers; -public abstract class ObjectPool { +public abstract class ObjectPool implements SchedulerLifecycle { private Queue pool; + private final int minSize; private final int maxSize; + private final long validationInterval; - private Scheduler.Worker schedulerWorker; + private final AtomicReference schedulerWorker; public ObjectPool() { this(0, 0, 67); @@ -50,31 +52,14 @@ public ObjectPool() { * When the number of objects is greater than maxIdle, too many instances will be removed. */ private ObjectPool(final int min, final int max, final long validationInterval) { + this.minSize = min; this.maxSize = max; + this.validationInterval = validationInterval; + this.schedulerWorker = new AtomicReference(); // initialize pool initialize(min); - schedulerWorker = Schedulers.computation().createWorker(); - schedulerWorker.schedulePeriodically(new Action0() { - - @Override - public void call() { - int size = pool.size(); - if (size < min) { - int sizeToBeAdded = max - size; - for (int i = 0; i < sizeToBeAdded; i++) { - pool.add(createObject()); - } - } else if (size > max) { - int sizeToBeRemoved = size - max; - for (int i = 0; i < sizeToBeRemoved; i++) { - // pool.pollLast(); - pool.poll(); - } - } - } - - }, validationInterval, validationInterval, TimeUnit.SECONDS); + start(); } /** @@ -109,10 +94,43 @@ public void returnObject(T object) { /** * Shutdown this pool. */ + @Override public void shutdown() { - schedulerWorker.unsubscribe(); + Worker w = schedulerWorker.getAndSet(null); + if (w != null) { + w.unsubscribe(); + } } + @Override + public void start() { + Worker w = Schedulers.computation().createWorker(); + if (schedulerWorker.compareAndSet(null, w)) { + w.schedulePeriodically(new Action0() { + + @Override + public void call() { + int size = pool.size(); + if (size < minSize) { + int sizeToBeAdded = maxSize - size; + for (int i = 0; i < sizeToBeAdded; i++) { + pool.add(createObject()); + } + } else if (size > maxSize) { + int sizeToBeRemoved = size - maxSize; + for (int i = 0; i < sizeToBeRemoved; i++) { + // pool.pollLast(); + pool.poll(); + } + } + } + + }, validationInterval, validationInterval, TimeUnit.SECONDS); + } else { + w.unsubscribe(); + } + } + /** * Creates a new object. * diff --git a/src/main/java/rx/internal/util/RxRingBuffer.java b/src/main/java/rx/internal/util/RxRingBuffer.java index f038b2deec..5f35c7f6e5 100644 --- a/src/main/java/rx/internal/util/RxRingBuffer.java +++ b/src/main/java/rx/internal/util/RxRingBuffer.java @@ -276,7 +276,8 @@ public static RxRingBuffer getSpmcInstance() { } public static final int SIZE = _size; - private static ObjectPool> SPSC_POOL = new ObjectPool>() { + /* Public so Schedulers can manage the lifecycle of the inner worker. */ + public static ObjectPool> SPSC_POOL = new ObjectPool>() { @Override protected SpscArrayQueue createObject() { @@ -285,7 +286,8 @@ protected SpscArrayQueue createObject() { }; - private static ObjectPool> SPMC_POOL = new ObjectPool>() { + /* Public so Schedulers can manage the lifecycle of the inner worker. */ + public static ObjectPool> SPMC_POOL = new ObjectPool>() { @Override protected SpmcArrayQueue createObject() { @@ -452,4 +454,4 @@ public boolean isUnsubscribed() { return queue == null; } -} +} \ No newline at end of file diff --git a/src/main/java/rx/schedulers/CachedThreadScheduler.java b/src/main/java/rx/schedulers/CachedThreadScheduler.java index f1cd815b64..6ef56a17cb 100644 --- a/src/main/java/rx/schedulers/CachedThreadScheduler.java +++ b/src/main/java/rx/schedulers/CachedThreadScheduler.java @@ -15,19 +15,16 @@ */ package rx.schedulers; -import rx.Scheduler; -import rx.Subscription; +import java.util.concurrent.*; +import java.util.concurrent.atomic.*; + +import rx.*; import rx.functions.Action0; -import rx.internal.schedulers.NewThreadWorker; -import rx.internal.schedulers.ScheduledAction; +import rx.internal.schedulers.*; import rx.internal.util.RxThreadFactory; -import rx.subscriptions.CompositeSubscription; -import rx.subscriptions.Subscriptions; - -import java.util.concurrent.*; -import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import rx.subscriptions.*; -/* package */final class CachedThreadScheduler extends Scheduler { +/* package */final class CachedThreadScheduler extends Scheduler implements SchedulerLifecycle { private static final String WORKER_THREAD_NAME_PREFIX = "RxCachedThreadScheduler-"; private static final RxThreadFactory WORKER_THREAD_FACTORY = new RxThreadFactory(WORKER_THREAD_NAME_PREFIX); @@ -36,31 +33,49 @@ private static final RxThreadFactory EVICTOR_THREAD_FACTORY = new RxThreadFactory(EVICTOR_THREAD_NAME_PREFIX); + private static final long KEEP_ALIVE_TIME = 60; + private static final TimeUnit KEEP_ALIVE_UNIT = TimeUnit.SECONDS; + + static final ThreadWorker SHUTDOWN_THREADWORKER; + static { + SHUTDOWN_THREADWORKER = new ThreadWorker(new RxThreadFactory("RxCachedThreadSchedulerShutdown-")); + SHUTDOWN_THREADWORKER.unsubscribe(); + } + private static final class CachedWorkerPool { private final long keepAliveTime; private final ConcurrentLinkedQueue expiringWorkerQueue; - private final ScheduledExecutorService evictExpiredWorkerExecutor; + private final CompositeSubscription allWorkers; + private final ScheduledExecutorService evictorService; + private final Future evictorTask; CachedWorkerPool(long keepAliveTime, TimeUnit unit) { - this.keepAliveTime = unit.toNanos(keepAliveTime); + this.keepAliveTime = unit != null ? unit.toNanos(keepAliveTime) : 0L; this.expiringWorkerQueue = new ConcurrentLinkedQueue(); - - evictExpiredWorkerExecutor = Executors.newScheduledThreadPool(1, EVICTOR_THREAD_FACTORY); - evictExpiredWorkerExecutor.scheduleWithFixedDelay( - new Runnable() { - @Override - public void run() { - evictExpiredWorkers(); - } - }, this.keepAliveTime, this.keepAliveTime, TimeUnit.NANOSECONDS - ); + this.allWorkers = new CompositeSubscription(); + + ScheduledExecutorService evictor = null; + Future task = null; + if (unit != null) { + evictor = Executors.newScheduledThreadPool(1, EVICTOR_THREAD_FACTORY); + NewThreadWorker.tryEnableCancelPolicy(evictor); + task = evictor.scheduleWithFixedDelay( + new Runnable() { + @Override + public void run() { + evictExpiredWorkers(); + } + }, this.keepAliveTime, this.keepAliveTime, TimeUnit.NANOSECONDS + ); + } + evictorService = evictor; + evictorTask = task; } - private static CachedWorkerPool INSTANCE = new CachedWorkerPool( - 60L, TimeUnit.SECONDS - ); - ThreadWorker get() { + if (allWorkers.isUnsubscribed()) { + return SHUTDOWN_THREADWORKER; + } while (!expiringWorkerQueue.isEmpty()) { ThreadWorker threadWorker = expiringWorkerQueue.poll(); if (threadWorker != null) { @@ -69,7 +84,9 @@ ThreadWorker get() { } // No cached worker found, so create a new one. - return new ThreadWorker(WORKER_THREAD_FACTORY); + ThreadWorker w = new ThreadWorker(WORKER_THREAD_FACTORY); + allWorkers.add(w); + return w; } void release(ThreadWorker threadWorker) { @@ -86,7 +103,7 @@ void evictExpiredWorkers() { for (ThreadWorker threadWorker : expiringWorkerQueue) { if (threadWorker.getExpirationTime() <= currentTimestamp) { if (expiringWorkerQueue.remove(threadWorker)) { - threadWorker.unsubscribe(); + allWorkers.remove(threadWorker); } } else { // Queue is ordered with the worker that will expire first in the beginning, so when we @@ -100,30 +117,79 @@ void evictExpiredWorkers() { long now() { return System.nanoTime(); } + + void shutdown() { + try { + if (evictorTask != null) { + evictorTask.cancel(true); + } + if (evictorService != null) { + evictorService.shutdownNow(); + } + } finally { + allWorkers.unsubscribe(); + } + } } + final AtomicReference pool; + + static final CachedWorkerPool NONE; + static { + NONE = new CachedWorkerPool(0, null); + NONE.shutdown(); + } + + public CachedThreadScheduler() { + this.pool = new AtomicReference(NONE); + start(); + } + + @Override + public void start() { + CachedWorkerPool update = new CachedWorkerPool(KEEP_ALIVE_TIME, KEEP_ALIVE_UNIT); + if (!pool.compareAndSet(NONE, update)) { + update.shutdown(); + } + } + @Override + public void shutdown() { + for (;;) { + CachedWorkerPool curr = pool.get(); + if (curr == NONE) { + return; + } + if (pool.compareAndSet(curr, NONE)) { + curr.shutdown(); + return; + } + } + } + @Override public Worker createWorker() { - return new EventLoopWorker(CachedWorkerPool.INSTANCE.get()); + return new EventLoopWorker(pool.get()); } private static final class EventLoopWorker extends Scheduler.Worker { private final CompositeSubscription innerSubscription = new CompositeSubscription(); + private final CachedWorkerPool pool; private final ThreadWorker threadWorker; @SuppressWarnings("unused") volatile int once; static final AtomicIntegerFieldUpdater ONCE_UPDATER = AtomicIntegerFieldUpdater.newUpdater(EventLoopWorker.class, "once"); - EventLoopWorker(ThreadWorker threadWorker) { - this.threadWorker = threadWorker; + EventLoopWorker(CachedWorkerPool pool) { + this.pool = pool; + this.threadWorker = pool.get(); } @Override public void unsubscribe() { if (ONCE_UPDATER.compareAndSet(this, 0, 1)) { // unsubscribe should be idempotent, so only do this once - CachedWorkerPool.INSTANCE.release(threadWorker); + pool.release(threadWorker); } innerSubscription.unsubscribe(); } @@ -168,4 +234,4 @@ public void setExpirationTime(long expirationTime) { this.expirationTime = expirationTime; } } -} +} \ No newline at end of file diff --git a/src/main/java/rx/schedulers/ExecutorScheduler.java b/src/main/java/rx/schedulers/ExecutorScheduler.java index ce9643cf2b..d447400184 100644 --- a/src/main/java/rx/schedulers/ExecutorScheduler.java +++ b/src/main/java/rx/schedulers/ExecutorScheduler.java @@ -20,7 +20,7 @@ import rx.*; import rx.functions.Action0; -import rx.internal.schedulers.ScheduledAction; +import rx.internal.schedulers.*; import rx.plugins.RxJavaPlugins; import rx.subscriptions.*; diff --git a/src/main/java/rx/schedulers/Schedulers.java b/src/main/java/rx/schedulers/Schedulers.java index 8ded001e0e..2376f0fa8a 100644 --- a/src/main/java/rx/schedulers/Schedulers.java +++ b/src/main/java/rx/schedulers/Schedulers.java @@ -16,7 +16,8 @@ package rx.schedulers; import rx.Scheduler; -import rx.internal.schedulers.EventLoopsScheduler; +import rx.internal.schedulers.*; +import rx.internal.util.RxRingBuffer; import rx.plugins.RxJavaPlugins; import java.util.concurrent.Executor; @@ -137,4 +138,52 @@ public static TestScheduler test() { public static Scheduler from(Executor executor) { return new ExecutorScheduler(executor); } -} + + /** + * Starts those standard Schedulers which support the SchedulerLifecycle interface. + *

The operation is idempotent and threadsafe. + */ + public static void start() { + Schedulers s = INSTANCE; + synchronized (s) { + if (s.computationScheduler instanceof SchedulerLifecycle) { + ((SchedulerLifecycle) s.computationScheduler).start(); + } + if (s.ioScheduler instanceof SchedulerLifecycle) { + ((SchedulerLifecycle) s.ioScheduler).start(); + } + if (s.newThreadScheduler instanceof SchedulerLifecycle) { + ((SchedulerLifecycle) s.newThreadScheduler).start(); + } + GenericScheduledExecutorService.INSTANCE.start(); + + RxRingBuffer.SPSC_POOL.start(); + + RxRingBuffer.SPMC_POOL.start(); + } + } + /** + * Shuts down those standard Schedulers which support the SchedulerLifecycle interface. + *

The operation is idempotent and threadsafe. + */ + public static void shutdown() { + Schedulers s = INSTANCE; + synchronized (s) { + if (s.computationScheduler instanceof SchedulerLifecycle) { + ((SchedulerLifecycle) s.computationScheduler).shutdown(); + } + if (s.ioScheduler instanceof SchedulerLifecycle) { + ((SchedulerLifecycle) s.ioScheduler).shutdown(); + } + if (s.newThreadScheduler instanceof SchedulerLifecycle) { + ((SchedulerLifecycle) s.newThreadScheduler).shutdown(); + } + + GenericScheduledExecutorService.INSTANCE.shutdown(); + + RxRingBuffer.SPSC_POOL.shutdown(); + + RxRingBuffer.SPMC_POOL.shutdown(); + } + } +} \ No newline at end of file diff --git a/src/test/java/rx/internal/schedulers/SchedulerLifecycleTest.java b/src/test/java/rx/internal/schedulers/SchedulerLifecycleTest.java new file mode 100644 index 0000000000..1d6b1b6abc --- /dev/null +++ b/src/test/java/rx/internal/schedulers/SchedulerLifecycleTest.java @@ -0,0 +1,127 @@ +package rx.internal.schedulers; + +import static org.junit.Assert.*; + +import java.util.*; +import java.util.concurrent.*; + +import org.junit.Test; + +import rx.Scheduler.Worker; +import rx.functions.Action0; +import rx.internal.util.RxRingBuffer; +import rx.schedulers.Schedulers; +import rx.subscriptions.CompositeSubscription; + +public class SchedulerLifecycleTest { + @Test + public void testShutdown() throws InterruptedException { + tryOutSchedulers(); + + System.out.println("testShutdown >> Giving time threads to spin-up"); + Thread.sleep(500); + + Set rxThreads = new HashSet(); + for (Thread t : Thread.getAllStackTraces().keySet()) { + if (t.getName().startsWith("Rx")) { + rxThreads.add(t); + } + } + Schedulers.shutdown(); + System.out.println("testShutdown >> Giving time to threads to stop"); + Thread.sleep(500); + + StringBuilder b = new StringBuilder(); + for (Thread t : rxThreads) { + if (t.isAlive()) { + b.append("Thread " + t + " failed to shutdown\r\n"); + for (StackTraceElement ste : t.getStackTrace()) { + b.append(" ").append(ste).append("\r\n"); + } + } + } + if (b.length() > 0) { + System.out.print(b); + System.out.println("testShutdown >> Restarting schedulers..."); + Schedulers.start(); // restart them anyways + fail("Rx Threads were still alive:\r\n" + b); + } + + System.out.println("testShutdown >> Restarting schedulers..."); + Schedulers.start(); + + tryOutSchedulers(); + } + + private void tryOutSchedulers() throws InterruptedException { + final CountDownLatch cdl = new CountDownLatch(4); + + final Action0 countAction = new Action0() { + @Override + public void call() { + cdl.countDown(); + } + }; + + CompositeSubscription csub = new CompositeSubscription(); + + try { + Worker w1 = Schedulers.computation().createWorker(); + csub.add(w1); + w1.schedule(countAction); + + Worker w2 = Schedulers.io().createWorker(); + csub.add(w2); + w2.schedule(countAction); + + Worker w3 = Schedulers.newThread().createWorker(); + csub.add(w3); + w3.schedule(countAction); + + GenericScheduledExecutorService.getInstance().execute(new Runnable() { + @Override + public void run() { + countAction.call(); + } + }); + + RxRingBuffer.getSpscInstance().release(); + + if (!cdl.await(3, TimeUnit.SECONDS)) { + fail("countAction was not run by every worker"); + } + } finally { + csub.unsubscribe(); + } + } + + @Test + public void testStartIdempotence() throws InterruptedException { + tryOutSchedulers(); + + System.out.println("testStartIdempotence >> giving some time"); + Thread.sleep(500); + + Set rxThreads = new HashSet(); + for (Thread t : Thread.getAllStackTraces().keySet()) { + if (t.getName().startsWith("Rx")) { + rxThreads.add(t); + System.out.println("testStartIdempotence >> " + t); + } + } + System.out.println("testStartIdempotence >> trying to start again"); + Schedulers.start(); + System.out.println("testStartIdempotence >> giving some time again"); + Thread.sleep(500); + + Set rxThreads2 = new HashSet(); + for (Thread t : Thread.getAllStackTraces().keySet()) { + if (t.getName().startsWith("Rx")) { + rxThreads2.add(t); + System.out.println("testStartIdempotence >>>> " + t); + } + } + + assertEquals(rxThreads, rxThreads2); + } +} \ No newline at end of file From bf1359fa04e3122d7d70f6af260d2be6ac3d2096 Mon Sep 17 00:00:00 2001 From: akarnokd Date: Wed, 30 Sep 2015 09:04:08 +0200 Subject: [PATCH 65/81] Hiding start(), moved test to compensate. --- src/main/java/rx/schedulers/Schedulers.java | 2 +- .../rx/{internal => }/schedulers/SchedulerLifecycleTest.java | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) rename src/test/java/rx/{internal => }/schedulers/SchedulerLifecycleTest.java (97%) diff --git a/src/main/java/rx/schedulers/Schedulers.java b/src/main/java/rx/schedulers/Schedulers.java index 2376f0fa8a..7dd8186616 100644 --- a/src/main/java/rx/schedulers/Schedulers.java +++ b/src/main/java/rx/schedulers/Schedulers.java @@ -143,7 +143,7 @@ public static Scheduler from(Executor executor) { * Starts those standard Schedulers which support the SchedulerLifecycle interface. *

The operation is idempotent and threadsafe. */ - public static void start() { + /* public testonly */ static void start() { Schedulers s = INSTANCE; synchronized (s) { if (s.computationScheduler instanceof SchedulerLifecycle) { diff --git a/src/test/java/rx/internal/schedulers/SchedulerLifecycleTest.java b/src/test/java/rx/schedulers/SchedulerLifecycleTest.java similarity index 97% rename from src/test/java/rx/internal/schedulers/SchedulerLifecycleTest.java rename to src/test/java/rx/schedulers/SchedulerLifecycleTest.java index 1d6b1b6abc..ce6d743cad 100644 --- a/src/test/java/rx/internal/schedulers/SchedulerLifecycleTest.java +++ b/src/test/java/rx/schedulers/SchedulerLifecycleTest.java @@ -1,4 +1,4 @@ -package rx.internal.schedulers; +package rx.schedulers; import static org.junit.Assert.*; @@ -9,6 +9,7 @@ import rx.Scheduler.Worker; import rx.functions.Action0; +import rx.internal.schedulers.GenericScheduledExecutorService; import rx.internal.util.RxRingBuffer; import rx.schedulers.Schedulers; import rx.subscriptions.CompositeSubscription; From 730be587d5fce689037d2cb4538e68cb04930f8b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Da=CC=81vid=20Karnok?= Date: Thu, 1 Oct 2015 20:40:02 +0200 Subject: [PATCH 66/81] pull back the Experimental/Beta of the changes until 1.1.x (+1 squashed commit) Squashed commits: [c6e43fc] 1.0.15. Beta/Deprecation of Subject state peeking methods. This should give users one release to prepare for the class structure changes. --- src/main/java/rx/subjects/AsyncSubject.java | 1 + src/main/java/rx/subjects/PublishSubject.java | 21 +++++++++++ src/main/java/rx/subjects/ReplaySubject.java | 2 ++ .../java/rx/subjects/SerializedSubject.java | 36 +++++++++++++++++++ src/main/java/rx/subjects/Subject.java | 14 ++++++++ 5 files changed, 74 insertions(+) diff --git a/src/main/java/rx/subjects/AsyncSubject.java b/src/main/java/rx/subjects/AsyncSubject.java index c186b1f78c..e3e508164f 100644 --- a/src/main/java/rx/subjects/AsyncSubject.java +++ b/src/main/java/rx/subjects/AsyncSubject.java @@ -203,6 +203,7 @@ public Throwable getThrowable() { } @Override @Experimental + @Deprecated @SuppressWarnings("unchecked") public T[] getValues(T[] a) { Object v = lastValue; diff --git a/src/main/java/rx/subjects/PublishSubject.java b/src/main/java/rx/subjects/PublishSubject.java index 1197048c3f..6ec0af1608 100644 --- a/src/main/java/rx/subjects/PublishSubject.java +++ b/src/main/java/rx/subjects/PublishSubject.java @@ -155,23 +155,44 @@ public Throwable getThrowable() { return null; } + /** + * {@inheritDoc} + * @deprecated this method is scheduled to be removed in the next release + */ @Override @Experimental + @Deprecated public boolean hasValue() { return false; } + + /** + * {@inheritDoc} + * @deprecated this method is scheduled to be removed in the next release + */ @Override @Experimental + @Deprecated public T getValue() { return null; } + /** + * {@inheritDoc} + * @deprecated this method is scheduled to be removed in the next release + */ @Override @Experimental + @Deprecated public Object[] getValues() { return new Object[0]; } + /** + * {@inheritDoc} + * @deprecated this method is scheduled to be removed in the next release + */ @Override @Experimental + @Deprecated public T[] getValues(T[] a) { if (a.length > 0) { a[0] = null; diff --git a/src/main/java/rx/subjects/ReplaySubject.java b/src/main/java/rx/subjects/ReplaySubject.java index c3779dac2d..f2230f4bba 100644 --- a/src/main/java/rx/subjects/ReplaySubject.java +++ b/src/main/java/rx/subjects/ReplaySubject.java @@ -1162,7 +1162,9 @@ public boolean hasValue() { public T[] getValues(T[] a) { return state.toArray(a); } + @Override + @Experimental public T getValue() { return state.latest(); } diff --git a/src/main/java/rx/subjects/SerializedSubject.java b/src/main/java/rx/subjects/SerializedSubject.java index edf4caeefe..6dd5a46592 100644 --- a/src/main/java/rx/subjects/SerializedSubject.java +++ b/src/main/java/rx/subjects/SerializedSubject.java @@ -69,38 +69,74 @@ public void onNext(T t) { public boolean hasObservers() { return actual.hasObservers(); } + + /** + * {@inheritDoc} + * @deprecated this method is scheduled to be removed in the next release + */ @Override @Experimental + @Deprecated public boolean hasCompleted() { return actual.hasCompleted(); } + /** + * {@inheritDoc} + * @deprecated this method is scheduled to be removed in the next release + */ @Override @Experimental + @Deprecated public boolean hasThrowable() { return actual.hasThrowable(); } + /** + * {@inheritDoc} + * @deprecated this method is scheduled to be removed in the next release + */ @Override @Experimental + @Deprecated public boolean hasValue() { return actual.hasValue(); } + /** + * {@inheritDoc} + * @deprecated this method is scheduled to be removed in the next release + */ @Override @Experimental + @Deprecated public Throwable getThrowable() { return actual.getThrowable(); } + /** + * {@inheritDoc} + * @deprecated this method is scheduled to be removed in the next release + */ @Override @Experimental + @Deprecated public T getValue() { return actual.getValue(); } + /** + * {@inheritDoc} + * @deprecated this method is scheduled to be removed in the next release + */ @Override @Experimental + @Deprecated public Object[] getValues() { return actual.getValues(); } + /** + * {@inheritDoc} + * @deprecated this method is scheduled to be removed in the next release + */ @Override @Experimental + @Deprecated public T[] getValues(T[] a) { return actual.getValues(a); } diff --git a/src/main/java/rx/subjects/Subject.java b/src/main/java/rx/subjects/Subject.java index 075dfe8e93..b220cc1b51 100644 --- a/src/main/java/rx/subjects/Subject.java +++ b/src/main/java/rx/subjects/Subject.java @@ -64,8 +64,10 @@ public final SerializedSubject toSerialized() { * * @return {@code true} if the subject has received a throwable through {@code onError}. * @since (If this graduates from being an Experimental class method, replace this parenthetical with the release number) + * @deprecated this method will be moved to each Subject class individually in the next release */ @Experimental + @Deprecated public boolean hasThrowable() { throw new UnsupportedOperationException(); } @@ -75,8 +77,10 @@ public boolean hasThrowable() { * * @return {@code true} if the subject completed normally via {@code onCompleted} * @since (If this graduates from being an Experimental class method, replace this parenthetical with the release number) + * @deprecated this method will be moved to each Subject class individually in the next release */ @Experimental + @Deprecated public boolean hasCompleted() { throw new UnsupportedOperationException(); } @@ -87,8 +91,10 @@ public boolean hasCompleted() { * @return the Throwable that terminated the Subject or {@code null} if the subject hasn't terminated yet or * if it terminated normally. * @since (If this graduates from being an Experimental class method, replace this parenthetical with the release number) + * @deprecated this method will be moved to each Subject class individually in the next release */ @Experimental + @Deprecated public Throwable getThrowable() { throw new UnsupportedOperationException(); } @@ -101,8 +107,10 @@ public Throwable getThrowable() { * * @return {@code true} if and only if the subject has some value but not an error * @since (If this graduates from being an Experimental class method, replace this parenthetical with the release number) + * @deprecated this method will be moved to each Subject class individually in the next release */ @Experimental + @Deprecated public boolean hasValue() { throw new UnsupportedOperationException(); } @@ -117,8 +125,10 @@ public boolean hasValue() { * @return the current value or {@code null} if the Subject doesn't have a value, has terminated with an * exception or has an actual {@code null} as a value. * @since (If this graduates from being an Experimental class method, replace this parenthetical with the release number) + * @deprecated this method will be moved to each Subject class individually in the next release */ @Experimental + @Deprecated public T getValue() { throw new UnsupportedOperationException(); } @@ -130,9 +140,11 @@ public T getValue() { * * @return a snapshot of the currently buffered non-terminal events. * @since (If this graduates from being an Experimental class method, replace this parenthetical with the release number) + * @deprecated this method will be moved to each Subject class individually in the next release */ @SuppressWarnings("unchecked") @Experimental + @Deprecated public Object[] getValues() { T[] r = getValues((T[])EMPTY_ARRAY); if (r == EMPTY_ARRAY) { @@ -152,8 +164,10 @@ public Object[] getValues() { * @param a the array to fill in * @return the array {@code a} if it had enough capacity or a new array containing the available values * @since (If this graduates from being an Experimental class method, replace this parenthetical with the release number) + * @deprecated this method will be moved to each Subject class individually in the next release */ @Experimental + @Deprecated public T[] getValues(T[] a) { throw new UnsupportedOperationException(); } From 29775c9f9775fb9f27538fc546f50b8936422845 Mon Sep 17 00:00:00 2001 From: Aaron Tull Date: Tue, 29 Sep 2015 10:25:12 -0700 Subject: [PATCH 67/81] Added warning to `Observable.doOnRequest` javadoc. --- src/main/java/rx/Observable.java | 22 ++++++++++++++-------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/src/main/java/rx/Observable.java b/src/main/java/rx/Observable.java index 0b50ef9268..a1f14d789b 100644 --- a/src/main/java/rx/Observable.java +++ b/src/main/java/rx/Observable.java @@ -4444,20 +4444,26 @@ public final void onNext(T args) { return lift(new OperatorDoOnEach(observer)); } - + /** - * Modifies the source {@code Observable} so that it invokes the given action when it receives a request for - * more items. + * Modifies the source {@code Observable} so that it invokes the given action when it receives a + * request for more items. + *

+ * Note: This operator is for tracing the internal behavior of back-pressure request + * patterns and generally intended for debugging use. *

- *
Scheduler:
- *
{@code doOnRequest} does not operate by default on a particular {@link Scheduler}.
+ *
Scheduler:
+ *
{@code doOnRequest} does not operate by default on a particular {@link Scheduler}.
*
* * @param onRequest - * the action that gets called when an observer requests items from this {@code Observable} + * the action that gets called when an observer requests items from this + * {@code Observable} * @return the source {@code Observable} modified so as to call this Action when appropriate - * @see ReactiveX operators documentation: Do - * @since (if this graduates from Experimental/Beta to supported, replace this parenthetical with the release number) + * @see ReactiveX operators + * documentation: Do + * @since (if this graduates from Experimental/Beta to supported, replace this parenthetical + * with the release number) */ @Beta public final Observable doOnRequest(final Action1 onRequest) { From 952223521d2d5f3f2eadacee9c346344f494892a Mon Sep 17 00:00:00 2001 From: akarnokd Date: Fri, 2 Oct 2015 09:06:46 +0200 Subject: [PATCH 68/81] DoOnEach: report both original exception and callback exception. This came up in a [Stackoverflow](http://stackoverflow.com/questions/32889008/do-operators-instead-of-a-whole-subscriber) answer. If the doOnError's callback or the doOnEach's onError method throws, any non-fatal exception replaced the original error which got lost. This PR will wrap them both into a CompositeException. 2.x note: since Java 8 supports `addSuppressed` all callbacks in this situation either attach to the original exception or the original exception is attached to the callback's exception. --- src/main/java/rx/Observable.java | 7 +++ .../internal/operators/OperatorDoOnEach.java | 7 ++- .../operators/OperatorDoOnEachTest.java | 48 +++++++++++++------ 3 files changed, 45 insertions(+), 17 deletions(-) diff --git a/src/main/java/rx/Observable.java b/src/main/java/rx/Observable.java index a1f14d789b..3678725ede 100644 --- a/src/main/java/rx/Observable.java +++ b/src/main/java/rx/Observable.java @@ -4362,6 +4362,10 @@ public final void onNext(T v) { /** * Modifies the source Observable so that it notifies an Observer for each item it emits. *

+ * In case the onError of the supplied observer throws, the downstream will receive a composite exception containing + * the original exception and the exception thrown by onError. If the onNext or the onCompleted methods + * of the supplied observer throws, the downstream will be terminated and wil receive this thrown exception. + *

* *

*
Scheduler:
@@ -4380,6 +4384,9 @@ public final Observable doOnEach(Observer observer) { /** * Modifies the source Observable so that it invokes an action if it calls {@code onError}. *

+ * In case the onError action throws, the downstream will receive a composite exception containing + * the original exception and the exception thrown by onError. + *

* *

*
Scheduler:
diff --git a/src/main/java/rx/internal/operators/OperatorDoOnEach.java b/src/main/java/rx/internal/operators/OperatorDoOnEach.java index 4b3e8d54cf..1e3a680dac 100644 --- a/src/main/java/rx/internal/operators/OperatorDoOnEach.java +++ b/src/main/java/rx/internal/operators/OperatorDoOnEach.java @@ -15,9 +15,11 @@ */ package rx.internal.operators; +import java.util.Arrays; + import rx.*; import rx.Observable.Operator; -import rx.exceptions.Exceptions; +import rx.exceptions.*; /** * Converts the elements of an observable sequence to the specified type. @@ -62,7 +64,8 @@ public void onError(Throwable e) { try { doOnEachObserver.onError(e); } catch (Throwable e2) { - Exceptions.throwOrReport(e2, observer); + Exceptions.throwIfFatal(e2); + observer.onError(new CompositeException(Arrays.asList(e, e2))); return; } observer.onError(e); diff --git a/src/test/java/rx/internal/operators/OperatorDoOnEachTest.java b/src/test/java/rx/internal/operators/OperatorDoOnEachTest.java index 2ad9a36828..3c4cf9f9bb 100644 --- a/src/test/java/rx/internal/operators/OperatorDoOnEachTest.java +++ b/src/test/java/rx/internal/operators/OperatorDoOnEachTest.java @@ -17,25 +17,19 @@ import static org.junit.Assert.*; import static org.mockito.Matchers.any; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; - -import org.junit.Before; -import org.junit.Test; -import org.mockito.Mock; -import org.mockito.MockitoAnnotations; - -import rx.Observable; -import rx.Observer; -import rx.Subscriber; -import rx.exceptions.OnErrorNotImplementedException; -import rx.functions.Action1; -import rx.functions.Func1; +import static org.mockito.Mockito.*; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; +import org.junit.*; +import org.mockito.*; + +import rx.*; +import rx.exceptions.*; +import rx.functions.*; +import rx.observers.TestSubscriber; + public class OperatorDoOnEachTest { @Mock @@ -201,4 +195,28 @@ public void call(Object o) { System.out.println("Received exception: " + e); } } + + @Test + public void testOnErrorThrows() { + TestSubscriber ts = TestSubscriber.create(); + + Observable.error(new TestException()) + .doOnError(new Action1() { + @Override + public void call(Throwable e) { + throw new TestException(); + } + }).subscribe(ts); + + ts.assertNoValues(); + ts.assertNotCompleted(); + ts.assertError(CompositeException.class); + + CompositeException ex = (CompositeException)ts.getOnErrorEvents().get(0); + + List exceptions = ex.getExceptions(); + assertEquals(2, exceptions.size()); + assertTrue(exceptions.get(0) instanceof TestException); + assertTrue(exceptions.get(1) instanceof TestException); + } } \ No newline at end of file From 53c523c196055de14b4574935f7a165318c5f85e Mon Sep 17 00:00:00 2001 From: Artem Zinnatullin Date: Sun, 20 Sep 2015 03:59:43 +0300 Subject: [PATCH 69/81] Remove unused private method from CachedObservable and make "state" final --- .../rx/internal/operators/CachedObservable.java | 13 +++---------- 1 file changed, 3 insertions(+), 10 deletions(-) diff --git a/src/main/java/rx/internal/operators/CachedObservable.java b/src/main/java/rx/internal/operators/CachedObservable.java index 0231c3590f..1995174eff 100644 --- a/src/main/java/rx/internal/operators/CachedObservable.java +++ b/src/main/java/rx/internal/operators/CachedObservable.java @@ -29,8 +29,9 @@ * @param the source element type */ public final class CachedObservable extends Observable { + /** The cache and replay state. */ - private CacheState state; + private final CacheState state; /** * Creates a cached Observable with a default capacity hint of 16. @@ -82,15 +83,7 @@ private CachedObservable(OnSubscribe onSubscribe, CacheState state) { /* public */ boolean hasObservers() { return state.producers.length != 0; } - - /** - * Returns the number of events currently cached. - * @return - */ - /* public */ int cachedEventCount() { - return state.size(); - } - + /** * Contains the active child producers and the values to replay. * From 79f2ae7cb5dd684d48d63c359fd78cfe27126986 Mon Sep 17 00:00:00 2001 From: Artem Zinnatullin Date: Sun, 20 Sep 2015 03:14:29 +0300 Subject: [PATCH 70/81] Safer error handling in BlockingOperatorToFuture --- .../rx/internal/operators/BlockingOperatorToFuture.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/main/java/rx/internal/operators/BlockingOperatorToFuture.java b/src/main/java/rx/internal/operators/BlockingOperatorToFuture.java index ee9a5fe314..29021405ca 100644 --- a/src/main/java/rx/internal/operators/BlockingOperatorToFuture.java +++ b/src/main/java/rx/internal/operators/BlockingOperatorToFuture.java @@ -118,8 +118,10 @@ public T get(long timeout, TimeUnit unit) throws InterruptedException, Execution } private T getValue() throws ExecutionException { - if (error.get() != null) { - throw new ExecutionException("Observable onError", error.get()); + final Throwable throwable = error.get(); + + if (throwable != null) { + throw new ExecutionException("Observable onError", throwable); } else if (cancelled) { // Contract of Future.get() requires us to throw this: throw new CancellationException("Subscription unsubscribed"); From c929e0d2424fa7cbc66a7e74f9b416930c8e2d8d Mon Sep 17 00:00:00 2001 From: Kevin Coughlin Date: Fri, 11 Sep 2015 15:17:23 -0400 Subject: [PATCH 71/81] Remove unnecessary onStart in OperatorGroupBy --- src/main/java/rx/internal/operators/OperatorGroupBy.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/main/java/rx/internal/operators/OperatorGroupBy.java b/src/main/java/rx/internal/operators/OperatorGroupBy.java index ffced4c923..02efb20f3f 100644 --- a/src/main/java/rx/internal/operators/OperatorGroupBy.java +++ b/src/main/java/rx/internal/operators/OperatorGroupBy.java @@ -259,9 +259,6 @@ public void call() { } }).unsafeSubscribe(new Subscriber(o) { - @Override - public void onStart() { - } @Override public void onCompleted() { o.onCompleted(); From 4cac9f5e9759ea3f2560ae58ac4ab04db63b1512 Mon Sep 17 00:00:00 2001 From: akarnokd Date: Sat, 3 Oct 2015 09:28:06 +0200 Subject: [PATCH 72/81] Eager concatMap --- src/main/java/rx/Observable.java | 406 ++++++++++++++++++ .../operators/OperatorEagerConcatMap.java | 308 +++++++++++++ .../util/atomic/SpscLinkedArrayQueue.java | 356 +++++++++++++++ .../operators/OperatorEagerConcatMapTest.java | 397 +++++++++++++++++ 4 files changed, 1467 insertions(+) create mode 100644 src/main/java/rx/internal/operators/OperatorEagerConcatMap.java create mode 100644 src/main/java/rx/internal/util/atomic/SpscLinkedArrayQueue.java create mode 100644 src/test/java/rx/internal/operators/OperatorEagerConcatMapTest.java diff --git a/src/main/java/rx/Observable.java b/src/main/java/rx/Observable.java index 3678725ede..ebe421a120 100644 --- a/src/main/java/rx/Observable.java +++ b/src/main/java/rx/Observable.java @@ -4559,6 +4559,412 @@ public final Observable doOnUnsubscribe(final Action0 unsubscribe) { return lift(new OperatorDoOnUnsubscribe(unsubscribe)); } + /** + * Concatenates up to 2 sources eagerly into a single stream of values. + * + *

+ * Eager concatenation means that once a Subscriber subscribes, all sources are subscribed to, their values fully buffered and + * they are drained in order, one after the previous completes. + * + *

+ *
Backpressure:
+ *
Backpressure is honored towards the downstream, however, due to the eagerness requirement, sources are subscribed to + * in unbounded mode and values queued up in an unbounded buffer.
+ *
Scheduler:
+ *
This method does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the value type + * @param o1 the first source + * @param o2 the second source + * @return + */ + @Experimental + @SuppressWarnings("unchecked") + public static Observable concatEager(Observable o1, Observable o2) { + return concatEager(Arrays.asList(o1, o2)); + } + + /** + * Concatenates up to 3 sources eagerly into a single stream of values. + * + *

+ * Eager concatenation means that once a Subscriber subscribes, all sources are subscribed to, their values fully buffered and + * they are drained in order, one after the previous completes. + * + *

+ *
Backpressure:
+ *
Backpressure is honored towards the downstream, however, due to the eagerness requirement, sources are subscribed to + * in unbounded mode and values queued up in an unbounded buffer.
+ *
Scheduler:
+ *
This method does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the value type + * @param o1 the first source + * @param o2 the second source + * @param o3 the third source + * @return + */ + @Experimental + @SuppressWarnings("unchecked") + public static Observable concatEager( + Observable o1, Observable o2, + Observable o3 + ) { + return concatEager(Arrays.asList(o1, o2, o3)); + } + + /** + * Concatenates up to 4 sources eagerly into a single stream of values. + * + *

+ * Eager concatenation means that once a Subscriber subscribes, all sources are subscribed to, their values fully buffered and + * they are drained in order, one after the previous completes. + * + *

+ *
Backpressure:
+ *
Backpressure is honored towards the downstream, however, due to the eagerness requirement, sources are subscribed to + * in unbounded mode and values queued up in an unbounded buffer.
+ *
Scheduler:
+ *
This method does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the value type + * @param o1 the first source + * @param o2 the second source + * @param o3 the third source + * @param o4 the fourth source + * @return + */ + @Experimental + @SuppressWarnings("unchecked") + public static Observable concatEager( + Observable o1, Observable o2, + Observable o3, Observable o4 + ) { + return concatEager(Arrays.asList(o1, o2, o3, o4)); + } + + /** + * Concatenates up to 5 sources eagerly into a single stream of values. + * + *

+ * Eager concatenation means that once a Subscriber subscribes, all sources are subscribed to, their values fully buffered and + * they are drained in order, one after the previous completes. + * + *

+ *
Backpressure:
+ *
Backpressure is honored towards the downstream, however, due to the eagerness requirement, sources are subscribed to + * in unbounded mode and values queued up in an unbounded buffer.
+ *
Scheduler:
+ *
This method does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the value type + * @param o1 the first source + * @param o2 the second source + * @param o3 the third source + * @param o4 the fourth source + * @param o5 the fifth source + * @return + */ + @Experimental + @SuppressWarnings("unchecked") + public static Observable concatEager( + Observable o1, Observable o2, + Observable o3, Observable o4, + Observable o5 + ) { + return concatEager(Arrays.asList(o1, o2, o3, o4, o5)); + } + + /** + * Concatenates up to 6 sources eagerly into a single stream of values. + * + *

+ * Eager concatenation means that once a Subscriber subscribes, all sources are subscribed to, their values fully buffered and + * they are drained in order, one after the previous completes. + * + *

+ *
Backpressure:
+ *
Backpressure is honored towards the downstream, however, due to the eagerness requirement, sources are subscribed to + * in unbounded mode and values queued up in an unbounded buffer.
+ *
Scheduler:
+ *
This method does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the value type + * @param o1 the first source + * @param o2 the second source + * @param o3 the third source + * @param o4 the fourth source + * @param o5 the fifth source + * @param o6 the sixth source + * @return + */ + @Experimental + @SuppressWarnings("unchecked") + public static Observable concatEager( + Observable o1, Observable o2, + Observable o3, Observable o4, + Observable o5, Observable o6 + ) { + return concatEager(Arrays.asList(o1, o2, o3, o4, o5, o6)); + } + + /** + * Concatenates up to 7 sources eagerly into a single stream of values. + * + *

+ * Eager concatenation means that once a Subscriber subscribes, all sources are subscribed to, their values fully buffered and + * they are drained in order, one after the previous completes. + * + *

+ *
Backpressure:
+ *
Backpressure is honored towards the downstream, however, due to the eagerness requirement, sources are subscribed to + * in unbounded mode and values queued up in an unbounded buffer.
+ *
Scheduler:
+ *
This method does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the value type + * @param o1 the first source + * @param o2 the second source + * @param o3 the third source + * @param o4 the fourth source + * @param o5 the fifth source + * @param o6 the sixth source + * @param o7 the seventh source + * @return + */ + @Experimental + @SuppressWarnings("unchecked") + public static Observable concatEager( + Observable o1, Observable o2, + Observable o3, Observable o4, + Observable o5, Observable o6, + Observable o7 + ) { + return concatEager(Arrays.asList(o1, o2, o3, o4, o5, o6, o7)); + } + + /** + * Concatenates up to 8 sources eagerly into a single stream of values. + * + *

+ * Eager concatenation means that once a Subscriber subscribes, all sources are subscribed to, their values fully buffered and + * they are drained in order, one after the previous completes. + * + *

+ *
Backpressure:
+ *
Backpressure is honored towards the downstream, however, due to the eagerness requirement, sources are subscribed to + * in unbounded mode and values queued up in an unbounded buffer.
+ *
Scheduler:
+ *
This method does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the value type + * @param o1 the first source + * @param o2 the second source + * @param o3 the third source + * @param o4 the fourth source + * @param o5 the fifth source + * @param o6 the sixth source + * @param o7 the seventh source + * @param o8 the eight source + * @return + */ + @Experimental + @SuppressWarnings("unchecked") + public static Observable concatEager( + Observable o1, Observable o2, + Observable o3, Observable o4, + Observable o5, Observable o6, + Observable o7, Observable o8 + ) { + return concatEager(Arrays.asList(o1, o2, o3, o4, o5, o6, o7, o8)); + } + + /** + * Concatenates up to 9 sources eagerly into a single stream of values. + * + *

+ * Eager concatenation means that once a Subscriber subscribes, all sources are subscribed to, their values fully buffered and + * they are drained in order, one after the previous completes. + * + *

+ *
Backpressure:
+ *
Backpressure is honored towards the downstream, however, due to the eagerness requirement, sources are subscribed to + * in unbounded mode and values queued up in an unbounded buffer.
+ *
Scheduler:
+ *
This method does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the value type + * @param o1 the first source + * @param o2 the second source + * @param o3 the third source + * @param o4 the fourth source + * @param o5 the fifth source + * @param o6 the sixth source + * @param o7 the seventh source + * @param o8 the eight source + * @param o9 the nine source + * @return + */ + @Experimental + @SuppressWarnings("unchecked") + public static Observable concatEager( + Observable o1, Observable o2, + Observable o3, Observable o4, + Observable o5, Observable o6, + Observable o7, Observable o8, + Observable o9 + ) { + return concatEager(Arrays.asList(o1, o2, o3, o4, o5, o6, o7, o8, o9)); + } + + /** + * Concatenates a sequence of Observables eagerly into a single stream of values. + * + *

+ * Eager concatenation means that once a Subscriber subscribes, all sources are subscribed to, their values fully buffered and + * they are drained in order, one after the previous completes. + * + *

+ *
Backpressure:
+ *
Backpressure is honored towards the downstream, however, due to the eagerness requirement, sources are subscribed to + * in unbounded mode and values queued up in an unbounded buffer.
+ *
Scheduler:
+ *
This method does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the value type + * @param sources a sequence of Observables that need to be eagerly concatenated + * @return + */ + @Experimental + @SuppressWarnings({ "unchecked", "rawtypes" }) + public static Observable concatEager(Iterable> sources) { + return Observable.from(sources).concatMapEager((Func1)UtilityFunctions.identity()); + } + + /** + * Concatenates a sequence of Observables eagerly into a single stream of values. + * + *

+ * Eager concatenation means that once a Subscriber subscribes, all sources are subscribed to, their values fully buffered and + * they are drained in order, one after the previous completes. + * + *

+ *
Backpressure:
+ *
Backpressure is honored towards the downstream, however, due to the eagerness requirement, sources are subscribed to + * in unbounded mode and values queued up in an unbounded buffer.
+ *
Scheduler:
+ *
This method does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the value type + * @param sources a sequence of Observables that need to be eagerly concatenated + * @param capacityHint hints about the number of expected source sequence values + * @return + */ + @Experimental + @SuppressWarnings({ "unchecked", "rawtypes" }) + public static Observable concatEager(Iterable> sources, int capacityHint) { + return Observable.from(sources).concatMapEager((Func1)UtilityFunctions.identity(), capacityHint); + } + + /** + * Concatenates an Observable sequence of Observables eagerly into a single stream of values. + * + *

+ * Eager concatenation means that once a Subscriber subscribes, all sources are subscribed to, their values fully buffered and + * they are drained in order, one after the previous completes. + * + *

+ *
Backpressure:
+ *
Backpressure is honored towards the downstream, however, due to the eagerness requirement, sources are subscribed to + * in unbounded mode and values queued up in an unbounded buffer.
+ *
Scheduler:
+ *
This method does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the value type + * @param sources a sequence of Observables that need to be eagerly concatenated + * @return + */ + @Experimental + @SuppressWarnings({ "unchecked", "rawtypes" }) + public static Observable concatEager(Observable> sources) { + return sources.concatMapEager((Func1)UtilityFunctions.identity()); + } + + /** + * Concatenates an Observable sequence of Observables eagerly into a single stream of values. + * + *

+ * Eager concatenation means that once a Subscriber subscribes, all sources are subscribed to, their values fully buffered and + * they are drained in order, one after the previous completes. + * + *

+ *
Backpressure:
+ *
Backpressure is honored towards the downstream, however, due to the eagerness requirement, sources are subscribed to + * in unbounded mode and values queued up in an unbounded buffer.
+ *
Scheduler:
+ *
This method does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the value type + * @param sources a sequence of Observables that need to be eagerly concatenated + * @param capacityHint hints about the number of expected source sequence values + * @return + */ + @Experimental + @SuppressWarnings({ "unchecked", "rawtypes" }) + public static Observable concatEager(Observable> sources, int capacityHint) { + return sources.concatMapEager((Func1)UtilityFunctions.identity(), capacityHint); + } + + /** + * Maps a sequence of values into Observables and concatenates these Observables eagerly into a single stream of values. + * + *

+ * Eager concatenation means that once a Subscriber subscribes, all sources are subscribed to, their values fully buffered and + * they are drained in order, one after the previous completes. + * + *

+ *
Backpressure:
+ *
Backpressure is honored towards the downstream, however, due to the eagerness requirement, sources are subscribed to + * in unbounded mode and values queued up in an unbounded buffer.
+ *
Scheduler:
+ *
This method does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the value type + * @param mapper the function that maps a sequence of values into a sequence of Observables that will bec eagerly concatenated + * @return + */ + @Experimental + public final Observable concatMapEager(Func1> mapper) { + return concatMapEager(mapper, RxRingBuffer.SIZE); + } + + /** + * Maps a sequence of values into Observables and concatenates these Observables eagerly into a single stream of values. + * + *

+ * Eager concatenation means that once a Subscriber subscribes, all sources are subscribed to, their values fully buffered and + * they are drained in order, one after the previous completes. + * + *

+ *
Backpressure:
+ *
Backpressure is honored towards the downstream, however, due to the eagerness requirement, sources are subscribed to + * in unbounded mode and values queued up in an unbounded buffer.
+ *
Scheduler:
+ *
This method does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the value type + * @param mapper the function that maps a sequence of values into a sequence of Observables that will bec eagerly concatenated + * @param capacityHint hints about the number of expected source sequence values + * @return + */ + @Experimental + public final Observable concatMapEager(Func1> mapper, int capacityHint) { + if (capacityHint < 1) { + throw new IllegalArgumentException("capacityHint > 0 required but it was " + capacityHint); + } + return lift(new OperatorEagerConcatMap(mapper, capacityHint)); + } + /** * Returns an Observable that emits the single item at a specified index in a sequence of emissions from a * source Observbable. diff --git a/src/main/java/rx/internal/operators/OperatorEagerConcatMap.java b/src/main/java/rx/internal/operators/OperatorEagerConcatMap.java new file mode 100644 index 0000000000..127f2fbd51 --- /dev/null +++ b/src/main/java/rx/internal/operators/OperatorEagerConcatMap.java @@ -0,0 +1,308 @@ +/** + * Copyright 2014 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.internal.operators; + +import java.util.*; +import java.util.concurrent.atomic.*; + +import rx.*; +import rx.Observable; +import rx.Observable.Operator; +import rx.exceptions.Exceptions; +import rx.functions.*; +import rx.internal.util.unsafe.SpscArrayQueue; +import rx.subscriptions.Subscriptions; + +public final class OperatorEagerConcatMap implements Operator { + final Func1> mapper; + final int bufferSize; + public OperatorEagerConcatMap(Func1> mapper, int bufferSize) { + this.mapper = mapper; + this.bufferSize = bufferSize; + } + + @Override + public Subscriber call(Subscriber t) { + EagerOuterSubscriber outer = new EagerOuterSubscriber(mapper, bufferSize, t); + outer.init(); + return outer; + } + + static final class EagerOuterProducer extends AtomicLong implements Producer { + /** */ + private static final long serialVersionUID = -657299606803478389L; + + final EagerOuterSubscriber parent; + + public EagerOuterProducer(EagerOuterSubscriber parent) { + this.parent = parent; + } + + @Override + public void request(long n) { + if (n < 0) { + throw new IllegalStateException("n >= 0 required but it was " + n); + } + + if (n > 0) { + BackpressureUtils.getAndAddRequest(this, n); + parent.drain(); + } + } + } + + static final class EagerOuterSubscriber extends Subscriber { + final Func1> mapper; + final int bufferSize; + final Subscriber actual; + + final LinkedList> subscribers; + + volatile boolean done; + Throwable error; + + volatile boolean cancelled; + + final AtomicInteger wip; + private EagerOuterProducer sharedProducer; + + public EagerOuterSubscriber(Func1> mapper, int bufferSize, + Subscriber actual) { + this.mapper = mapper; + this.bufferSize = bufferSize; + this.actual = actual; + this.subscribers = new LinkedList>(); + this.wip = new AtomicInteger(); + } + + void init() { + sharedProducer = new EagerOuterProducer(this); + add(Subscriptions.create(new Action0() { + @Override + public void call() { + cancelled = true; + if (wip.getAndIncrement() == 0) { + cleanup(); + } + } + })); + actual.add(this); + actual.setProducer(sharedProducer); + } + + void cleanup() { + List list; + synchronized (subscribers) { + list = new ArrayList(subscribers); + subscribers.clear(); + } + + for (Subscription s : list) { + s.unsubscribe(); + } + } + + @Override + public void onNext(T t) { + Observable observable; + + try { + observable = mapper.call(t); + } catch (Throwable e) { + Exceptions.throwOrReport(e, actual, t); + return; + } + + EagerInnerSubscriber inner = new EagerInnerSubscriber(this, bufferSize); + if (cancelled) { + return; + } + synchronized (subscribers) { + if (cancelled) { + return; + } + subscribers.add(inner); + } + if (cancelled) { + return; + } + observable.unsafeSubscribe(inner); + drain(); + } + + @Override + public void onError(Throwable e) { + error = e; + done = true; + drain(); + } + + @Override + public void onCompleted() { + done = true; + drain(); + } + + void drain() { + if (wip.getAndIncrement() != 0) { + return; + } + int missed = 1; + + final AtomicLong requested = sharedProducer; + final Subscriber actualSubscriber = this.actual; + + for (;;) { + + if (cancelled) { + cleanup(); + return; + } + + EagerInnerSubscriber innerSubscriber; + + boolean outerDone = done; + synchronized (subscribers) { + innerSubscriber = subscribers.peek(); + } + boolean empty = innerSubscriber == null; + + if (outerDone) { + Throwable error = this.error; + if (error != null) { + cleanup(); + actualSubscriber.onError(error); + return; + } else + if (empty) { + actualSubscriber.onCompleted(); + return; + } + } + + if (!empty) { + long requestedAmount = requested.get(); + long emittedAmount = 0L; + boolean unbounded = requestedAmount == Long.MAX_VALUE; + + Queue innerQueue = innerSubscriber.queue; + boolean innerDone = false; + + + for (;;) { + outerDone = innerSubscriber.done; + R v = innerQueue.peek(); + empty = v == null; + + if (outerDone) { + Throwable innerError = innerSubscriber.error; + if (innerError != null) { + cleanup(); + actualSubscriber.onError(innerError); + return; + } else + if (empty) { + synchronized (subscribers) { + subscribers.poll(); + } + innerSubscriber.unsubscribe(); + innerDone = true; + break; + } + } + + if (empty) { + break; + } + + if (requestedAmount == 0L) { + break; + } + + innerQueue.poll(); + + try { + actualSubscriber.onNext(v); + } catch (Throwable ex) { + Exceptions.throwOrReport(ex, actualSubscriber, v); + return; + } + + requestedAmount--; + emittedAmount--; + } + + if (emittedAmount != 0L) { + if (!unbounded) { + requested.addAndGet(emittedAmount); + } + if (!innerDone) { + innerSubscriber.requestMore(-emittedAmount); + } + } + + if (innerDone) { + continue; + } + } + + missed = wip.addAndGet(-missed); + if (missed == 0) { + return; + } + } + } + } + + static final class EagerInnerSubscriber extends Subscriber { + final EagerOuterSubscriber parent; + final Queue queue; + + volatile boolean done; + Throwable error; + + public EagerInnerSubscriber(EagerOuterSubscriber parent, int bufferSize) { + super(); + this.parent = parent; + this.queue = new SpscArrayQueue(bufferSize); + request(bufferSize); + } + + @Override + public void onNext(T t) { + queue.offer(t); + parent.drain(); + } + + @Override + public void onError(Throwable e) { + error = e; + done = true; + parent.drain(); + } + + @Override + public void onCompleted() { + done = true; + parent.drain(); + } + + void requestMore(long n) { + request(n); + } + } +} diff --git a/src/main/java/rx/internal/util/atomic/SpscLinkedArrayQueue.java b/src/main/java/rx/internal/util/atomic/SpscLinkedArrayQueue.java new file mode 100644 index 0000000000..5a00430b96 --- /dev/null +++ b/src/main/java/rx/internal/util/atomic/SpscLinkedArrayQueue.java @@ -0,0 +1,356 @@ +/** + * Copyright 2014 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.internal.util.atomic; + +import java.util.*; +import java.util.concurrent.atomic.*; + +import rx.internal.util.unsafe.Pow2; + + +/* + * The code was inspired by the similarly named JCTools class: + * https://github.com/JCTools/JCTools/blob/master/jctools-core/src/main/java/org/jctools/queues/atomic + */ + +/** + * A single-producer single-consumer array-backed queue which can allocate new arrays in case the consumer is slower + * than the producer. + */ +public final class SpscLinkedArrayQueue implements Queue { + static final int MAX_LOOK_AHEAD_STEP = Integer.getInteger("jctools.spsc.max.lookahead.step", 4096); + protected volatile long producerIndex; + @SuppressWarnings("rawtypes") + static final AtomicLongFieldUpdater PRODUCER_INDEX = + AtomicLongFieldUpdater.newUpdater(SpscLinkedArrayQueue.class, "producerIndex"); + protected int producerLookAheadStep; + protected long producerLookAhead; + protected int producerMask; + protected AtomicReferenceArray producerBuffer; + protected int consumerMask; + protected AtomicReferenceArray consumerBuffer; + protected volatile long consumerIndex; + @SuppressWarnings("rawtypes") + static final AtomicLongFieldUpdater CONSUMER_INDEX = + AtomicLongFieldUpdater.newUpdater(SpscLinkedArrayQueue.class, "consumerIndex"); + private static final Object HAS_NEXT = new Object(); + + public SpscLinkedArrayQueue(final int bufferSize) { + int p2capacity = Pow2.roundToPowerOfTwo(bufferSize); + int mask = p2capacity - 1; + AtomicReferenceArray buffer = new AtomicReferenceArray(p2capacity + 1); + producerBuffer = buffer; + producerMask = mask; + adjustLookAheadStep(p2capacity); + consumerBuffer = buffer; + consumerMask = mask; + producerLookAhead = mask - 1; // we know it's all empty to start with + soProducerIndex(0L); + } + + /** + * {@inheritDoc} + *

+ * This implementation is correct for single producer thread use only. + */ + @Override + public final boolean offer(final T e) { + // local load of field to avoid repeated loads after volatile reads + final AtomicReferenceArray buffer = producerBuffer; + final long index = lpProducerIndex(); + final int mask = producerMask; + final int offset = calcWrappedOffset(index, mask); + if (index < producerLookAhead) { + return writeToQueue(buffer, e, index, offset); + } else { + final int lookAheadStep = producerLookAheadStep; + // go around the buffer or resize if full (unless we hit max capacity) + int lookAheadElementOffset = calcWrappedOffset(index + lookAheadStep, mask); + if (null == lvElement(buffer, lookAheadElementOffset)) {// LoadLoad + producerLookAhead = index + lookAheadStep - 1; // joy, there's plenty of room + return writeToQueue(buffer, e, index, offset); + } else if (null == lvElement(buffer, calcWrappedOffset(index + 1, mask))) { // buffer is not full + return writeToQueue(buffer, e, index, offset); + } else { + resize(buffer, index, offset, e, mask); // add a buffer and link old to new + return true; + } + } + } + + private boolean writeToQueue(final AtomicReferenceArray buffer, final T e, final long index, final int offset) { + soProducerIndex(index + 1);// this ensures atomic write of long on 32bit platforms + soElement(buffer, offset, e);// StoreStore + return true; + } + + private void resize(final AtomicReferenceArray oldBuffer, final long currIndex, final int offset, final T e, + final long mask) { + final int capacity = oldBuffer.length(); + final AtomicReferenceArray newBuffer = new AtomicReferenceArray(capacity); + producerBuffer = newBuffer; + producerLookAhead = currIndex + mask - 1; + soProducerIndex(currIndex + 1);// this ensures correctness on 32bit platforms + soElement(newBuffer, offset, e);// StoreStore + soNext(oldBuffer, newBuffer); + soElement(oldBuffer, offset, HAS_NEXT); // new buffer is visible after element is + // inserted + } + + private void soNext(AtomicReferenceArray curr, AtomicReferenceArray next) { + soElement(curr, calcDirectOffset(curr.length() - 1), next); + } + @SuppressWarnings("unchecked") + private AtomicReferenceArray lvNext(AtomicReferenceArray curr) { + return (AtomicReferenceArray)lvElement(curr, calcDirectOffset(curr.length() - 1)); + } + /** + * {@inheritDoc} + *

+ * This implementation is correct for single consumer thread use only. + */ + @SuppressWarnings("unchecked") + @Override + public final T poll() { + // local load of field to avoid repeated loads after volatile reads + final AtomicReferenceArray buffer = consumerBuffer; + final long index = lpConsumerIndex(); + final int mask = consumerMask; + final int offset = calcWrappedOffset(index, mask); + final Object e = lvElement(buffer, offset);// LoadLoad + boolean isNextBuffer = e == HAS_NEXT; + if (null != e && !isNextBuffer) { + soConsumerIndex(index + 1);// this ensures correctness on 32bit platforms + soElement(buffer, offset, null);// StoreStore + return (T) e; + } else if (isNextBuffer) { + return newBufferPoll(lvNext(buffer), index, mask); + } + + return null; + } + + @SuppressWarnings("unchecked") + private T newBufferPoll(AtomicReferenceArray nextBuffer, final long index, final int mask) { + consumerBuffer = nextBuffer; + final int offsetInNew = calcWrappedOffset(index, mask); + final T n = (T) lvElement(nextBuffer, offsetInNew);// LoadLoad + if (null == n) { + return null; + } else { + soConsumerIndex(index + 1);// this ensures correctness on 32bit platforms + soElement(nextBuffer, offsetInNew, null);// StoreStore + return n; + } + } + + /** + * {@inheritDoc} + *

+ * This implementation is correct for single consumer thread use only. + */ + @SuppressWarnings("unchecked") + @Override + public final T peek() { + final AtomicReferenceArray buffer = consumerBuffer; + final long index = lpConsumerIndex(); + final int mask = consumerMask; + final int offset = calcWrappedOffset(index, mask); + final Object e = lvElement(buffer, offset);// LoadLoad + if (e == HAS_NEXT) { + return newBufferPeek(lvNext(buffer), index, mask); + } + + return (T) e; + } + + @Override + public void clear() { + while (poll() != null || !isEmpty()); + } + + @SuppressWarnings("unchecked") + private T newBufferPeek(AtomicReferenceArray nextBuffer, final long index, final int mask) { + consumerBuffer = nextBuffer; + final int offsetInNew = calcWrappedOffset(index, mask); + return (T) lvElement(nextBuffer, offsetInNew);// LoadLoad + } + + @Override + public final int size() { + /* + * It is possible for a thread to be interrupted or reschedule between the read of the producer and + * consumer indices, therefore protection is required to ensure size is within valid range. In the + * event of concurrent polls/offers to this method the size is OVER estimated as we read consumer + * index BEFORE the producer index. + */ + long after = lvConsumerIndex(); + while (true) { + final long before = after; + final long currentProducerIndex = lvProducerIndex(); + after = lvConsumerIndex(); + if (before == after) { + return (int) (currentProducerIndex - after); + } + } + } + + @Override + public boolean isEmpty() { + return lvProducerIndex() == lvConsumerIndex(); + } + + private void adjustLookAheadStep(int capacity) { + producerLookAheadStep = Math.min(capacity / 4, MAX_LOOK_AHEAD_STEP); + } + + private long lvProducerIndex() { + return producerIndex; + } + + private long lvConsumerIndex() { + return consumerIndex; + } + + private long lpProducerIndex() { + return producerIndex; + } + + private long lpConsumerIndex() { + return consumerIndex; + } + + private void soProducerIndex(long v) { + PRODUCER_INDEX.lazySet(this, v); + } + + private void soConsumerIndex(long v) { + CONSUMER_INDEX.lazySet(this, v); + } + + private static final int calcWrappedOffset(long index, int mask) { + return calcDirectOffset((int)index & mask); + } + private static final int calcDirectOffset(int index) { + return index; + } + private static final void soElement(AtomicReferenceArray buffer, int offset, Object e) { + buffer.lazySet(offset, e); + } + + private static final Object lvElement(AtomicReferenceArray buffer, int offset) { + return buffer.get(offset); + } + + @Override + public final Iterator iterator() { + throw new UnsupportedOperationException(); + } + + @Override + public boolean contains(Object o) { + throw new UnsupportedOperationException(); + } + + @Override + public Object[] toArray() { + throw new UnsupportedOperationException(); + } + + @Override + public E[] toArray(E[] a) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean remove(Object o) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean containsAll(Collection c) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean addAll(Collection c) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean removeAll(Collection c) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean retainAll(Collection c) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean add(T e) { + throw new UnsupportedOperationException(); + } + + @Override + public T remove() { + throw new UnsupportedOperationException(); + } + + @Override + public T element() { + throw new UnsupportedOperationException(); + } + + /** + * Offer two elements at the same time. + *

Don't use the regular offer() with this at all! + * @param first + * @param second + * @return + */ + public boolean offer(T first, T second) { + final AtomicReferenceArray buffer = producerBuffer; + final long p = producerIndex; + final int m = producerMask; + + int pi = calcWrappedOffset(p + 2, m); + + if (null == lvElement(buffer, pi)) { + pi = calcWrappedOffset(p, m); + soElement(buffer, pi + 1, second); + soProducerIndex(p + 2); + soElement(buffer, pi, first); + } else { + final int capacity = buffer.length(); + final AtomicReferenceArray newBuffer = new AtomicReferenceArray(capacity); + producerBuffer = newBuffer; + + pi = calcWrappedOffset(p, m); + soElement(newBuffer, pi + 1, second);// StoreStore + soElement(newBuffer, pi, first); + soNext(buffer, newBuffer); + + soProducerIndex(p + 2);// this ensures correctness on 32bit platforms + + soElement(buffer, pi, HAS_NEXT); // new buffer is visible after element is + } + + return true; + } +} + diff --git a/src/test/java/rx/internal/operators/OperatorEagerConcatMapTest.java b/src/test/java/rx/internal/operators/OperatorEagerConcatMapTest.java new file mode 100644 index 0000000000..8c7bd3d9e4 --- /dev/null +++ b/src/test/java/rx/internal/operators/OperatorEagerConcatMapTest.java @@ -0,0 +1,397 @@ +/** + * Copyright 2014 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.internal.operators; + +import java.util.concurrent.atomic.*; + +import org.junit.*; + +import rx.Observable; +import rx.exceptions.TestException; +import rx.functions.*; +import rx.internal.util.RxRingBuffer; +import rx.observers.TestSubscriber; +import rx.schedulers.Schedulers; +import rx.subjects.PublishSubject; + +public class OperatorEagerConcatMapTest { + TestSubscriber ts; + TestSubscriber tsBp; + + Func1> toJust = new Func1>() { + @Override + public Observable call(Integer t) { + return Observable.just(t); + } + }; + + Func1> toRange = new Func1>() { + @Override + public Observable call(Integer t) { + return Observable.range(t, 2); + } + }; + + @Before + public void before() { + ts = new TestSubscriber(); + tsBp = new TestSubscriber(0L); + } + + @Test + public void testSimple() { + Observable.range(1, 100).concatMapEager(toJust).subscribe(ts); + + ts.assertNoErrors(); + ts.assertValueCount(100); + ts.assertCompleted(); + } + + @Test + public void testSimple2() { + Observable.range(1, 100).concatMapEager(toRange).subscribe(ts); + + ts.assertNoErrors(); + ts.assertValueCount(200); + ts.assertCompleted(); + } + + @Test + public void testEagerness2() { + final AtomicInteger count = new AtomicInteger(); + Observable source = Observable.just(1).doOnNext(new Action1() { + @Override + public void call(Integer t) { + count.getAndIncrement(); + } + }); + + Observable.concatEager(source, source).subscribe(tsBp); + + Assert.assertEquals(2, count.get()); + tsBp.assertNoErrors(); + tsBp.assertNotCompleted(); + tsBp.assertNoValues(); + + tsBp.requestMore(Long.MAX_VALUE); + + tsBp.assertValueCount(count.get()); + tsBp.assertNoErrors(); + tsBp.assertCompleted(); + } + + @Test + public void testEagerness3() { + final AtomicInteger count = new AtomicInteger(); + Observable source = Observable.just(1).doOnNext(new Action1() { + @Override + public void call(Integer t) { + count.getAndIncrement(); + } + }); + + Observable.concatEager(source, source, source).subscribe(tsBp); + + Assert.assertEquals(3, count.get()); + tsBp.assertNoErrors(); + tsBp.assertNotCompleted(); + tsBp.assertNoValues(); + + tsBp.requestMore(Long.MAX_VALUE); + + tsBp.assertValueCount(count.get()); + tsBp.assertNoErrors(); + tsBp.assertCompleted(); + } + + @Test + public void testEagerness4() { + final AtomicInteger count = new AtomicInteger(); + Observable source = Observable.just(1).doOnNext(new Action1() { + @Override + public void call(Integer t) { + count.getAndIncrement(); + } + }); + + Observable.concatEager(source, source, source, source).subscribe(tsBp); + + Assert.assertEquals(4, count.get()); + tsBp.assertNoErrors(); + tsBp.assertNotCompleted(); + tsBp.assertNoValues(); + + tsBp.requestMore(Long.MAX_VALUE); + + tsBp.assertValueCount(count.get()); + tsBp.assertNoErrors(); + tsBp.assertCompleted(); + } + + @Test + public void testEagerness5() { + final AtomicInteger count = new AtomicInteger(); + Observable source = Observable.just(1).doOnNext(new Action1() { + @Override + public void call(Integer t) { + count.getAndIncrement(); + } + }); + + Observable.concatEager(source, source, source, source, source).subscribe(tsBp); + + Assert.assertEquals(5, count.get()); + tsBp.assertNoErrors(); + tsBp.assertNotCompleted(); + tsBp.assertNoValues(); + + tsBp.requestMore(Long.MAX_VALUE); + + tsBp.assertValueCount(count.get()); + tsBp.assertNoErrors(); + tsBp.assertCompleted(); + } + + @Test + public void testEagerness6() { + final AtomicInteger count = new AtomicInteger(); + Observable source = Observable.just(1).doOnNext(new Action1() { + @Override + public void call(Integer t) { + count.getAndIncrement(); + } + }); + + Observable.concatEager(source, source, source, source, source, source).subscribe(tsBp); + + Assert.assertEquals(6, count.get()); + tsBp.assertNoErrors(); + tsBp.assertNotCompleted(); + tsBp.assertNoValues(); + + tsBp.requestMore(Long.MAX_VALUE); + + tsBp.assertValueCount(count.get()); + tsBp.assertNoErrors(); + tsBp.assertCompleted(); + } + + @Test + public void testEagerness7() { + final AtomicInteger count = new AtomicInteger(); + Observable source = Observable.just(1).doOnNext(new Action1() { + @Override + public void call(Integer t) { + count.getAndIncrement(); + } + }); + + Observable.concatEager(source, source, source, source, source, source, source).subscribe(tsBp); + + Assert.assertEquals(7, count.get()); + tsBp.assertNoErrors(); + tsBp.assertNotCompleted(); + tsBp.assertNoValues(); + + tsBp.requestMore(Long.MAX_VALUE); + + tsBp.assertValueCount(count.get()); + tsBp.assertNoErrors(); + tsBp.assertCompleted(); + } + + @Test + public void testEagerness8() { + final AtomicInteger count = new AtomicInteger(); + Observable source = Observable.just(1).doOnNext(new Action1() { + @Override + public void call(Integer t) { + count.getAndIncrement(); + } + }); + + Observable.concatEager(source, source, source, source, source, source, source, source).subscribe(tsBp); + + Assert.assertEquals(8, count.get()); + tsBp.assertNoErrors(); + tsBp.assertNotCompleted(); + tsBp.assertNoValues(); + + tsBp.requestMore(Long.MAX_VALUE); + + tsBp.assertValueCount(count.get()); + tsBp.assertNoErrors(); + tsBp.assertCompleted(); + } + + @Test + public void testEagerness9() { + final AtomicInteger count = new AtomicInteger(); + Observable source = Observable.just(1).doOnNext(new Action1() { + @Override + public void call(Integer t) { + count.getAndIncrement(); + } + }); + + Observable.concatEager(source, source, source, source, source, source, source, source, source).subscribe(tsBp); + + Assert.assertEquals(9, count.get()); + tsBp.assertNoErrors(); + tsBp.assertNotCompleted(); + tsBp.assertNoValues(); + + tsBp.requestMore(Long.MAX_VALUE); + + tsBp.assertValueCount(count.get()); + tsBp.assertNoErrors(); + tsBp.assertCompleted(); + } + + @Test + public void testMainError() { + Observable.error(new TestException()).concatMapEager(toJust).subscribe(ts); + + ts.assertNoValues(); + ts.assertError(TestException.class); + ts.assertNotCompleted(); + } + + @Test + public void testInnerError() { + Observable.concatEager(Observable.just(1), Observable.error(new TestException())).subscribe(ts); + + ts.assertValue(1); + ts.assertError(TestException.class); + ts.assertNotCompleted(); + } + + @Test + public void testInnerEmpty() { + Observable.concatEager(Observable.empty(), Observable.empty()).subscribe(ts); + + ts.assertNoValues(); + ts.assertNoErrors(); + ts.assertCompleted(); + } + + @Test + public void testMapperThrows() { + Observable.just(1).concatMapEager(new Func1>() { + @Override + public Observable call(Integer t) { + throw new TestException(); + } + }).subscribe(ts); + + ts.assertNoValues(); + ts.assertNotCompleted(); + ts.assertError(TestException.class); + } + + @Test(expected = IllegalArgumentException.class) + public void testInvalidCapacityHint() { + Observable.just(1).concatMapEager(toJust, 0); + } + + @Test + public void testBackpressure() { + Observable.concatEager(Observable.just(1), Observable.just(1)).subscribe(tsBp); + + tsBp.assertNoErrors(); + tsBp.assertNoValues(); + tsBp.assertNotCompleted(); + + tsBp.requestMore(1); + tsBp.assertValue(1); + tsBp.assertNoErrors(); + tsBp.assertNotCompleted(); + + tsBp.requestMore(1); + tsBp.assertValues(1, 1); + tsBp.assertNoErrors(); + tsBp.assertCompleted(); + } + + @Test + public void testAsynchronousRun() { + Observable.range(1, 2).concatMapEager(new Func1>() { + @Override + public Observable call(Integer t) { + return Observable.range(1, 1000).subscribeOn(Schedulers.computation()); + } + }).observeOn(Schedulers.newThread()).subscribe(ts); + + ts.awaitTerminalEvent(); + ts.assertNoErrors(); + ts.assertValueCount(2000); + } + + @Test + public void testReentrantWork() { + final PublishSubject subject = PublishSubject.create(); + + final AtomicBoolean once = new AtomicBoolean(); + + subject.concatMapEager(new Func1>() { + @Override + public Observable call(Integer t) { + return Observable.just(t); + } + }) + .doOnNext(new Action1() { + @Override + public void call(Integer t) { + if (once.compareAndSet(false, true)) { + subject.onNext(2); + } + } + }) + .subscribe(ts); + + subject.onNext(1); + + ts.assertNoErrors(); + ts.assertNotCompleted(); + ts.assertValues(1, 2); + } + + @Test + public void testPrefetchIsBounded() { + final AtomicInteger count = new AtomicInteger(); + + TestSubscriber ts = TestSubscriber.create(0); + + Observable.just(1).concatMapEager(new Func1>() { + @Override + public Observable call(Integer t) { + return Observable.range(1, RxRingBuffer.SIZE * 2) + .doOnNext(new Action1() { + @Override + public void call(Integer t) { + count.getAndIncrement(); + } + }); + } + }).subscribe(ts); + + ts.assertNoErrors(); + ts.assertNoValues(); + ts.assertNotCompleted(); + Assert.assertEquals(RxRingBuffer.SIZE, count.get()); + } +} From 3bafb53f424f4211c34938acf6dfb58cf1ed1e61 Mon Sep 17 00:00:00 2001 From: Victor Vu Date: Tue, 15 Sep 2015 21:04:20 -0600 Subject: [PATCH 73/81] Make BlockingOperatorToIterator exert backpressure. --- .../operators/BlockingOperatorToIterator.java | 115 ++++++++++-------- .../BlockingOperatorToIteratorTest.java | 42 +++++++ 2 files changed, 106 insertions(+), 51 deletions(-) diff --git a/src/main/java/rx/internal/operators/BlockingOperatorToIterator.java b/src/main/java/rx/internal/operators/BlockingOperatorToIterator.java index 6f631a211d..7070436c42 100644 --- a/src/main/java/rx/internal/operators/BlockingOperatorToIterator.java +++ b/src/main/java/rx/internal/operators/BlockingOperatorToIterator.java @@ -23,7 +23,6 @@ import rx.Notification; import rx.Observable; import rx.Subscriber; -import rx.Subscription; import rx.exceptions.Exceptions; /** @@ -47,68 +46,82 @@ private BlockingOperatorToIterator() { * @return the iterator that could be used to iterate over the elements of the observable. */ public static Iterator toIterator(Observable source) { - final BlockingQueue> notifications = new LinkedBlockingQueue>(); + SubscriberIterator subscriber = new SubscriberIterator(); // using subscribe instead of unsafeSubscribe since this is a BlockingObservable "final subscribe" - final Subscription subscription = source.materialize().subscribe(new Subscriber>() { - @Override - public void onCompleted() { - // ignore - } + source.materialize().subscribe(subscriber); + return subscriber; + } - @Override - public void onError(Throwable e) { - notifications.offer(Notification.createOnError(e)); - } + public static final class SubscriberIterator + extends Subscriber> implements Iterator { - @Override - public void onNext(Notification args) { - notifications.offer(args); - } - }); + private final BlockingQueue> notifications; + private Notification buf; - return new Iterator() { - private Notification buf; + public SubscriberIterator() { + this.notifications = new LinkedBlockingQueue>(); + this.buf = null; + } - @Override - public boolean hasNext() { - if (buf == null) { - buf = take(); - } - if (buf.isOnError()) { - throw Exceptions.propagate(buf.getThrowable()); - } - return !buf.isOnCompleted(); + @Override + public void onStart() { + request(0); + } + + @Override + public void onCompleted() { + // ignore + } + + @Override + public void onError(Throwable e) { + notifications.offer(Notification.createOnError(e)); + } + + @Override + public void onNext(Notification args) { + notifications.offer(args); + } + + @Override + public boolean hasNext() { + if (buf == null) { + request(1); + buf = take(); + } + if (buf.isOnError()) { + throw Exceptions.propagate(buf.getThrowable()); } + return !buf.isOnCompleted(); + } - @Override - public T next() { - if (hasNext()) { - T result = buf.getValue(); - buf = null; - return result; - } - throw new NoSuchElementException(); + @Override + public T next() { + if (hasNext()) { + T result = buf.getValue(); + buf = null; + return result; } + throw new NoSuchElementException(); + } - private Notification take() { - try { - Notification poll = notifications.poll(); - if (poll != null) { - return poll; - } - return notifications.take(); - } catch (InterruptedException e) { - subscription.unsubscribe(); - throw Exceptions.propagate(e); + private Notification take() { + try { + Notification poll = notifications.poll(); + if (poll != null) { + return poll; } + return notifications.take(); + } catch (InterruptedException e) { + unsubscribe(); + throw Exceptions.propagate(e); } + } - @Override - public void remove() { - throw new UnsupportedOperationException("Read-only iterator"); - } - }; + @Override + public void remove() { + throw new UnsupportedOperationException("Read-only iterator"); + } } - } diff --git a/src/test/java/rx/internal/operators/BlockingOperatorToIteratorTest.java b/src/test/java/rx/internal/operators/BlockingOperatorToIteratorTest.java index db92042f1a..d8cf0aa9ed 100644 --- a/src/test/java/rx/internal/operators/BlockingOperatorToIteratorTest.java +++ b/src/test/java/rx/internal/operators/BlockingOperatorToIteratorTest.java @@ -81,4 +81,46 @@ public void call(Subscriber subscriber) { System.out.println(string); } } + + @Test + public void testIteratorExertBackpressure() { + final Counter src = new Counter(); + + Observable obs = Observable.from(new Iterable() { + @Override + public Iterator iterator() { + return src; + } + }); + + Iterator it = toIterator(obs); + while (it.hasNext()) { + // Correct backpressure should cause this interleaved behavior. + int i = it.next(); + assertEquals(i + 1, src.count); + } + } + + public static final class Counter implements Iterator { + public int count; + + public Counter() { + this.count = 0; + } + + @Override + public boolean hasNext() { + return count < 5; + } + + @Override + public Integer next() { + return count++; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + } } From 782b0e78e8fe38aa86bde40b5e3da55809c0420f Mon Sep 17 00:00:00 2001 From: Victor Vu Date: Wed, 16 Sep 2015 02:20:50 -0600 Subject: [PATCH 74/81] Request data in batches. --- .../operators/BlockingOperatorToIterator.java | 13 ++++++++++--- .../BlockingOperatorToIteratorTest.java | 18 +++++++++++------- 2 files changed, 21 insertions(+), 10 deletions(-) diff --git a/src/main/java/rx/internal/operators/BlockingOperatorToIterator.java b/src/main/java/rx/internal/operators/BlockingOperatorToIterator.java index 7070436c42..899aaffacb 100644 --- a/src/main/java/rx/internal/operators/BlockingOperatorToIterator.java +++ b/src/main/java/rx/internal/operators/BlockingOperatorToIterator.java @@ -24,6 +24,7 @@ import rx.Observable; import rx.Subscriber; import rx.exceptions.Exceptions; +import rx.internal.util.RxRingBuffer; /** * Returns an Iterator that iterates over all items emitted by a specified Observable. @@ -56,17 +57,19 @@ public static Iterator toIterator(Observable source) { public static final class SubscriberIterator extends Subscriber> implements Iterator { + static final int LIMIT = 3 * RxRingBuffer.SIZE / 4; + private final BlockingQueue> notifications; private Notification buf; + private int received; public SubscriberIterator() { this.notifications = new LinkedBlockingQueue>(); - this.buf = null; } @Override public void onStart() { - request(0); + request(RxRingBuffer.SIZE); } @Override @@ -87,8 +90,12 @@ public void onNext(Notification args) { @Override public boolean hasNext() { if (buf == null) { - request(1); buf = take(); + received++; + if (received >= LIMIT) { + request(received); + received = 0; + } } if (buf.isOnError()) { throw Exceptions.propagate(buf.getThrowable()); diff --git a/src/test/java/rx/internal/operators/BlockingOperatorToIteratorTest.java b/src/test/java/rx/internal/operators/BlockingOperatorToIteratorTest.java index d8cf0aa9ed..4ed030660b 100644 --- a/src/test/java/rx/internal/operators/BlockingOperatorToIteratorTest.java +++ b/src/test/java/rx/internal/operators/BlockingOperatorToIteratorTest.java @@ -26,6 +26,8 @@ import rx.Observable.OnSubscribe; import rx.Subscriber; import rx.exceptions.TestException; +import rx.internal.operators.BlockingOperatorToIterator.SubscriberIterator; +import rx.internal.util.RxRingBuffer; public class BlockingOperatorToIteratorTest { @@ -96,26 +98,28 @@ public Iterator iterator() { Iterator it = toIterator(obs); while (it.hasNext()) { // Correct backpressure should cause this interleaved behavior. + // We first request RxRingBuffer.SIZE. Then in increments of + // SubscriberIterator.LIMIT. int i = it.next(); - assertEquals(i + 1, src.count); + int expected = i - (i % SubscriberIterator.LIMIT) + RxRingBuffer.SIZE; + expected = Math.min(expected, Counter.MAX); + + assertEquals(expected, src.count); } } public static final class Counter implements Iterator { + static final int MAX = 5 * RxRingBuffer.SIZE; public int count; - public Counter() { - this.count = 0; - } - @Override public boolean hasNext() { - return count < 5; + return count < MAX; } @Override public Integer next() { - return count++; + return ++count; } @Override From 2d0ff46c052e275b8d7ca00ccaced75533e94a0e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?D=C3=A1vid=20Karnok?= Date: Thu, 8 Oct 2015 18:17:53 +0200 Subject: [PATCH 75/81] BlockingObservable + subscribe methods. --- .../rx/observables/BlockingObservable.java | 256 +++++++++++++++++- .../observables/BlockingObservableTest.java | 164 +++++++++-- 2 files changed, 386 insertions(+), 34 deletions(-) diff --git a/src/main/java/rx/observables/BlockingObservable.java b/src/main/java/rx/observables/BlockingObservable.java index 7eced68981..805e217bbe 100644 --- a/src/main/java/rx/observables/BlockingObservable.java +++ b/src/main/java/rx/observables/BlockingObservable.java @@ -15,23 +15,19 @@ */ package rx.observables; -import java.util.Iterator; -import java.util.NoSuchElementException; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.Future; +import java.util.*; +import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicReference; +import rx.*; import rx.Observable; -import rx.Subscriber; -import rx.Subscription; -import rx.functions.Action1; -import rx.functions.Func1; -import rx.internal.operators.BlockingOperatorLatest; -import rx.internal.operators.BlockingOperatorMostRecent; -import rx.internal.operators.BlockingOperatorNext; -import rx.internal.operators.BlockingOperatorToFuture; -import rx.internal.operators.BlockingOperatorToIterator; +import rx.Observer; +import rx.annotations.Experimental; +import rx.exceptions.OnErrorNotImplementedException; +import rx.functions.*; +import rx.internal.operators.*; import rx.internal.util.UtilityFunctions; +import rx.subscriptions.Subscriptions; /** * {@code BlockingObservable} is a variety of {@link Observable} that provides blocking operators. It can be @@ -83,12 +79,16 @@ public static BlockingObservable from(final Observable o) { * need the {@link Subscriber#onCompleted()} or {@link Subscriber#onError(Throwable)} methods. If the * underlying Observable terminates with an error, rather than calling {@code onError}, this method will * throw an exception. - * + * + *

The difference between this method and {@link #subscribe(Action1)} is that the {@code onNext} action + * is executed on the emission thread instead of the current thread. + * * @param onNext * the {@link Action1} to invoke for each item emitted by the {@code BlockingObservable} * @throws RuntimeException * if an error occurs * @see ReactiveX documentation: Subscribe + * @see #subscribe(Action1) */ public void forEach(final Action1 onNext) { final CountDownLatch latch = new CountDownLatch(1); @@ -477,4 +477,232 @@ private void awaitForComplete(CountDownLatch latch, Subscription subscription) { throw new RuntimeException("Interrupted while waiting for subscription to complete.", e); } } + + /** + * Runs the source observable to a terminal event, ignoring any values and rethrowing any exception. + */ + @Experimental + public void run() { + final CountDownLatch cdl = new CountDownLatch(1); + final Throwable[] error = { null }; + Subscription s = o.subscribe(new Subscriber() { + @Override + public void onNext(T t) { + + } + @Override + public void onError(Throwable e) { + error[0] = e; + cdl.countDown(); + } + + @Override + public void onCompleted() { + cdl.countDown(); + } + }); + + awaitForComplete(cdl, s); + Throwable e = error[0]; + if (e != null) { + if (e instanceof RuntimeException) { + throw (RuntimeException)e; + } else { + throw new RuntimeException(e); + } + } + } + + /** + * Subscribes to the source and calls back the Observer methods on the current thread. + * @param observer the observer to call event methods on + */ + @Experimental + public void subscribe(Observer observer) { + final NotificationLite nl = NotificationLite.instance(); + final BlockingQueue queue = new LinkedBlockingQueue(); + + Subscription s = o.subscribe(new Subscriber() { + @Override + public void onNext(T t) { + queue.offer(nl.next(t)); + } + @Override + public void onError(Throwable e) { + queue.offer(nl.error(e)); + } + @Override + public void onCompleted() { + queue.offer(nl.completed()); + } + }); + + try { + for (;;) { + Object o = queue.poll(); + if (o == null) { + o = queue.take(); + } + if (nl.accept(observer, o)) { + return; + } + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + observer.onError(e); + } finally { + s.unsubscribe(); + } + } + + /** Constant to indicate the onStart method should be called. */ + private static final Object ON_START = new Object(); + + /** Constant indicating the setProducer method should be called. */ + private static final Object SET_PRODUCER = new Object(); + + /** Indicates an unsubscripton happened */ + private static final Object UNSUBSCRIBE = new Object(); + + /** + * Subscribes to the source and calls the Subscriber methods on the current thread. + *

+ * The unsubscription and backpressure is composed through. + * @param subscriber the subscriber to forward events and calls to in the current thread + */ + @Experimental + public void subscribe(Subscriber subscriber) { + final NotificationLite nl = NotificationLite.instance(); + final BlockingQueue queue = new LinkedBlockingQueue(); + final Producer[] theProducer = { null }; + + Subscriber s = new Subscriber() { + @Override + public void onNext(T t) { + queue.offer(nl.next(t)); + } + @Override + public void onError(Throwable e) { + queue.offer(nl.error(e)); + } + @Override + public void onCompleted() { + queue.offer(nl.completed()); + } + + @Override + public void setProducer(Producer p) { + theProducer[0] = p; + queue.offer(SET_PRODUCER); + } + + @Override + public void onStart() { + queue.offer(ON_START); + } + }; + + subscriber.add(s); + subscriber.add(Subscriptions.create(new Action0() { + @Override + public void call() { + queue.offer(UNSUBSCRIBE); + } + })); + + o.subscribe(s); + + try { + for (;;) { + if (subscriber.isUnsubscribed()) { + break; + } + Object o = queue.poll(); + if (o == null) { + o = queue.take(); + } + if (subscriber.isUnsubscribed() || o == UNSUBSCRIBE) { + break; + } + if (o == ON_START) { + subscriber.onStart(); + } else + if (o == SET_PRODUCER) { + subscriber.setProducer(theProducer[0]); + } else + if (nl.accept(subscriber, o)) { + return; + } + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + subscriber.onError(e); + } finally { + s.unsubscribe(); + } + } + + /** + * Runs the source observable to a terminal event, ignoring any values and rethrowing any exception. + */ + @Experimental + public void subscribe() { + run(); + } + + /** + * Subscribes to the source and calls the given action on the current thread and rethrows any exception wrapped + * into OnErrorNotImplementedException. + * + *

The difference between this method and {@link #forEach(Action1)} is that the + * action is always executed on the current thread. + * + * @param onNext the callback action for each source value + * @see #forEach(Action1) + */ + @Experimental + public void subscribe(final Action1 onNext) { + subscribe(onNext, new Action1() { + @Override + public void call(Throwable t) { + throw new OnErrorNotImplementedException(t); + } + }, Actions.empty()); + } + + /** + * Subscribes to the source and calls the given actions on the current thread. + * @param onNext the callback action for each source value + * @param onError the callback action for an error event + */ + @Experimental + public void subscribe(final Action1 onNext, final Action1 onError) { + subscribe(onNext, onError, Actions.empty()); + } + + /** + * Subscribes to the source and calls the given actions on the current thread. + * @param onNext the callback action for each source value + * @param onError the callback action for an error event + * @param onCompleted the callback action for the completion event. + */ + @Experimental + public void subscribe(final Action1 onNext, final Action1 onError, final Action0 onCompleted) { + subscribe(new Observer() { + @Override + public void onNext(T t) { + onNext.call(t); + } + + @Override + public void onError(Throwable e) { + onError.call(e); + } + + @Override + public void onCompleted() { + onCompleted.call(); + } + }); + } } diff --git a/src/test/java/rx/observables/BlockingObservableTest.java b/src/test/java/rx/observables/BlockingObservableTest.java index 4328461d80..72963f76ae 100644 --- a/src/test/java/rx/observables/BlockingObservableTest.java +++ b/src/test/java/rx/observables/BlockingObservableTest.java @@ -15,34 +15,25 @@ */ package rx.observables; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.mockito.Mock; -import org.mockito.MockitoAnnotations; - -import java.util.Iterator; -import java.util.NoSuchElementException; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; +import static org.junit.Assert.*; + +import java.io.IOException; +import java.util.*; +import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicReference; +import org.junit.*; +import org.mockito.*; + import rx.Observable; import rx.Observable.OnSubscribe; import rx.Subscriber; -import rx.exceptions.TestException; -import rx.functions.Action0; -import rx.functions.Action1; -import rx.functions.Func1; +import rx.exceptions.*; +import rx.functions.*; +import rx.observers.TestSubscriber; import rx.schedulers.Schedulers; import rx.subscriptions.Subscriptions; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - public class BlockingObservableTest { @Mock @@ -641,4 +632,137 @@ private InterruptedException getInterruptedExceptionOrNull() { } + @Test + public void testRun() { + Observable.just(1).observeOn(Schedulers.computation()).toBlocking().run(); + } + + @Test(expected = TestException.class) + public void testRunException() { + Observable.error(new TestException()).observeOn(Schedulers.computation()).toBlocking().run(); + } + + @Test + public void testRunIOException() { + try { + Observable.error(new IOException()).observeOn(Schedulers.computation()).toBlocking().run(); + fail("No exception thrown"); + } catch (RuntimeException ex) { + if (ex.getCause() instanceof IOException) { + return; + } + fail("Bad exception type: " + ex + ", " + ex.getCause()); + } + } + + @Test + public void testSubscriberBackpressure() { + TestSubscriber ts = new TestSubscriber() { + @Override + public void onStart() { + request(2); + } + + @Override + public void onNext(Integer t) { + super.onNext(t); + unsubscribe(); + } + }; + + Observable.range(1, 10).observeOn(Schedulers.computation()).toBlocking().subscribe(ts); + + ts.assertNoErrors(); + ts.assertNotCompleted(); + ts.assertValue(1); + } + + @Test(expected = OnErrorNotImplementedException.class) + public void testOnErrorNotImplemented() { + Observable.error(new TestException()).observeOn(Schedulers.computation()).toBlocking().subscribe(Actions.empty()); + } + + @Test + public void testSubscribeCallback1() { + final boolean[] valueReceived = { false }; + Observable.just(1).observeOn(Schedulers.computation()).toBlocking().subscribe(new Action1() { + @Override + public void call(Integer t) { + valueReceived[0] = true; + assertEquals((Integer)1, t); + } + }); + + assertTrue(valueReceived[0]); + } + + @Test + public void testSubscribeCallback2() { + final boolean[] received = { false }; + Observable.error(new TestException()).observeOn(Schedulers.computation()).toBlocking() + .subscribe(new Action1() { + @Override + public void call(Object t) { + fail("Value emitted: " + t); + } + }, new Action1() { + @Override + public void call(Throwable t) { + received[0] = true; + assertEquals(TestException.class, t.getClass()); + } + }); + + assertTrue(received[0]); + } + + @Test + public void testSubscribeCallback3() { + final boolean[] received = { false, false }; + Observable.just(1).observeOn(Schedulers.computation()).toBlocking().subscribe(new Action1() { + @Override + public void call(Integer t) { + received[0] = true; + assertEquals((Integer)1, t); + } + }, new Action1() { + @Override + public void call(Throwable t) { + t.printStackTrace(); + fail("Exception received!"); + } + }, new Action0() { + @Override + public void call() { + received[1] = true; + } + }); + + assertTrue(received[0]); + assertTrue(received[1]); + } + @Test + public void testSubscribeCallback3Error() { + final TestSubscriber ts = TestSubscriber.create(); + Observable.error(new TestException()).observeOn(Schedulers.computation()).toBlocking().subscribe(new Action1() { + @Override + public void call(Object t) { + ts.onNext(t); + } + }, new Action1() { + @Override + public void call(Throwable t) { + ts.onError(t); + } + }, new Action0() { + @Override + public void call() { + ts.onCompleted(); + } + }); + + ts.assertNoValues(); + ts.assertNotCompleted(); + ts.assertError(TestException.class); + } } From 7aef4f86b0e739a5fe996beecd5c6996597f5b5b Mon Sep 17 00:00:00 2001 From: George Campbell Date: Thu, 8 Oct 2015 10:21:52 -0700 Subject: [PATCH 76/81] Update README.md Slight change to make the distinction between `@Beta` and `@Experimental` explicit and meaningful. --- README.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index 5dcb2199a8..aca2675593 100644 --- a/README.md +++ b/README.md @@ -36,15 +36,15 @@ Patch 1.x.y increments (such as 1.0.0 -> 1.0.1, 1.3.1 -> 1.3.2, etc) will occur #### @Beta -APIs marked with the `@Beta` annotation at the class or method level are subject to change. They can be modified in any way, or even removed, at any time. If your code is a library itself (i.e. it is used on the CLASSPATH of users outside your own control), you should not use beta APIs, unless you repackage them (e.g. using ProGuard, shading, etc). +APIs marked with the `@Beta` annotation at the class or method level are subject to change. They can be modified in any way, or even removed in any major or minor release but not in a patch release. If your code is a library itself (i.e. it is used on the CLASSPATH of users outside your own control), you should not use beta APIs, unless you repackage them (e.g. using ProGuard, shading, etc). #### @Experimental -APIs marked with the `@Experimental` annotation at the class or method level will almost certainly change. They can be modified in any way, or even removed, at any time. You should not use or rely on them in any production code. They are purely to allow broad testing and feedback. +APIs marked with the `@Experimental` annotation at the class or method level will almost certainly change. They can be modified in any way, or even removed in any major, minor or, patch release. You should not use or rely on them in any production code. They are purely to allow broad testing and feedback. #### @Deprecated -APIs marked with the `@Deprecated` annotation at the class or method level will remain supported until the next major release but it is recommended to stop using them. +APIs marked with the `@Deprecated` annotation at the class or method level will remain supported until the next major release but it is recommended to stop using them. APIs marked with `@Beta` and `@Experimental` will be marked as deprecated for at least one minor release before they removed in a minor or patch release respectively. #### rx.internal.* From 324514a2beab19190d9eafffdd36fecbda03c10a Mon Sep 17 00:00:00 2001 From: Aaron Tull Date: Thu, 8 Oct 2015 12:05:18 -0700 Subject: [PATCH 77/81] Renaming Observable#x to Observable#extend --- src/main/java/rx/Observable.java | 2 +- src/test/java/rx/ObservableConversionTest.java | 4 ++-- src/test/java/rx/ObservableTests.java | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/main/java/rx/Observable.java b/src/main/java/rx/Observable.java index ebe421a120..02142ec6ce 100644 --- a/src/main/java/rx/Observable.java +++ b/src/main/java/rx/Observable.java @@ -119,7 +119,7 @@ public interface Operator extends Func1, Subscriber< * @since (if this graduates from Experimental/Beta to supported, replace this parenthetical with the release number) */ @Experimental - public R x(Func1, ? extends R> conversion) { + public R extend(Func1, ? extends R> conversion) { return conversion.call(new OnSubscribe() { @Override public void call(Subscriber subscriber) { diff --git a/src/test/java/rx/ObservableConversionTest.java b/src/test/java/rx/ObservableConversionTest.java index 543c44780b..31880ea599 100644 --- a/src/test/java/rx/ObservableConversionTest.java +++ b/src/test/java/rx/ObservableConversionTest.java @@ -155,7 +155,7 @@ public void onNext(String t) { }}); List crewOfBattlestarGalactica = Arrays.asList(new Object[] {"William Adama", "Laura Roslin", "Lee Adama", new Cylon()}); Observable.from(crewOfBattlestarGalactica) - .x(new ConvertToCylonDetector()) + .extend(new ConvertToCylonDetector()) .beep(new Func1(){ @Override public Boolean call(Object t) { @@ -199,7 +199,7 @@ public Integer call(Integer k) { return i + k; }}); }}) - .x(new Func1, ConcurrentLinkedQueue>() { + .extend(new Func1, ConcurrentLinkedQueue>() { @Override public ConcurrentLinkedQueue call(OnSubscribe onSubscribe) { final ConcurrentLinkedQueue q = new ConcurrentLinkedQueue(); diff --git a/src/test/java/rx/ObservableTests.java b/src/test/java/rx/ObservableTests.java index 55e43896d3..d59e8c41a9 100644 --- a/src/test/java/rx/ObservableTests.java +++ b/src/test/java/rx/ObservableTests.java @@ -1161,7 +1161,7 @@ public void testForEachWithNull() { public void testExtend() { final TestSubscriber subscriber = new TestSubscriber(); final Object value = new Object(); - Observable.just(value).x(new Func1,Object>(){ + Observable.just(value).extend(new Func1,Object>(){ @Override public Object call(OnSubscribe onSubscribe) { onSubscribe.call(subscriber); From 50c22340a2f6c091bfe6be1735f40c6884d5ed7c Mon Sep 17 00:00:00 2001 From: Artem Zinnatullin Date: Thu, 8 Oct 2015 06:08:03 +0300 Subject: [PATCH 78/81] Add Single.doOnError() --- src/main/java/rx/Single.java | 37 +++++++++++++++++ src/test/java/rx/SingleTest.java | 69 ++++++++++++++++++++++++++++++++ 2 files changed, 106 insertions(+) diff --git a/src/main/java/rx/Single.java b/src/main/java/rx/Single.java index 7fbf369b79..4324d32acf 100644 --- a/src/main/java/rx/Single.java +++ b/src/main/java/rx/Single.java @@ -31,6 +31,7 @@ import rx.functions.Func8; import rx.functions.Func9; import rx.internal.operators.OnSubscribeToObservableFuture; +import rx.internal.operators.OperatorDoOnEach; import rx.internal.operators.OperatorMap; import rx.internal.operators.OperatorObserveOn; import rx.internal.operators.OperatorOnErrorReturn; @@ -1789,4 +1790,40 @@ public final Single zipWith(Single other, Func2 + * In case the onError action throws, the downstream will receive a composite exception containing + * the original exception and the exception thrown by onError. + *

+ * + *

+ *
Scheduler:
+ *
{@code doOnError} does not operate by default on a particular {@link Scheduler}.
+ *
+ * + * @param onError + * the action to invoke if the source {@link Single} calls {@code onError} + * @return the source {@link Single} with the side-effecting behavior applied + * @see ReactiveX operators documentation: Do + */ + @Experimental + public final Single doOnError(final Action1 onError) { + Observer observer = new Observer() { + @Override + public void onCompleted() { + } + + @Override + public void onError(Throwable e) { + onError.call(e); + } + + @Override + public void onNext(T t) { + } + }; + + return lift(new OperatorDoOnEach(observer)); + } } diff --git a/src/test/java/rx/SingleTest.java b/src/test/java/rx/SingleTest.java index 1efd1ae5a7..7d8fe2dc22 100644 --- a/src/test/java/rx/SingleTest.java +++ b/src/test/java/rx/SingleTest.java @@ -13,8 +13,13 @@ package rx; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyZeroInteractions; import java.util.Arrays; import java.util.concurrent.CountDownLatch; @@ -26,7 +31,9 @@ import org.junit.Test; import rx.Single.OnSubscribe; +import rx.exceptions.CompositeException; import rx.functions.Action0; +import rx.functions.Action1; import rx.functions.Func1; import rx.functions.Func2; import rx.observers.TestSubscriber; @@ -461,4 +468,66 @@ public void testToObservable() { ts.assertValue("a"); ts.assertCompleted(); } + + @Test + public void doOnErrorShouldNotCallActionIfNoErrorHasOccurred() { + Action1 action = mock(Action1.class); + + TestSubscriber testSubscriber = new TestSubscriber(); + + Single + .just("value") + .doOnError(action) + .subscribe(testSubscriber); + + testSubscriber.assertValue("value"); + testSubscriber.assertNoErrors(); + + verifyZeroInteractions(action); + } + + @Test + public void doOnErrorShouldCallActionIfErrorHasOccurred() { + Action1 action = mock(Action1.class); + + TestSubscriber testSubscriber = new TestSubscriber(); + + Throwable error = new IllegalStateException(); + + Single + .error(error) + .doOnError(action) + .subscribe(testSubscriber); + + testSubscriber.assertNoValues(); + testSubscriber.assertError(error); + + verify(action).call(error); + } + + @Test + public void doOnErrorShouldThrowCompositeExceptionIfOnErrorActionThrows() { + Action1 action = mock(Action1.class); + + + Throwable error = new RuntimeException(); + Throwable exceptionFromOnErrorAction = new IllegalStateException(); + doThrow(exceptionFromOnErrorAction).when(action).call(error); + + TestSubscriber testSubscriber = new TestSubscriber(); + + Single + .error(error) + .doOnError(action) + .subscribe(testSubscriber); + + testSubscriber.assertNoValues(); + CompositeException compositeException = (CompositeException) testSubscriber.getOnErrorEvents().get(0); + + assertEquals(2, compositeException.getExceptions().size()); + assertSame(error, compositeException.getExceptions().get(0)); + assertSame(exceptionFromOnErrorAction, compositeException.getExceptions().get(1)); + + verify(action).call(error); + } } From 6634212b8641fad8786ecc49ec1a2367f7d54811 Mon Sep 17 00:00:00 2001 From: Artem Zinnatullin Date: Thu, 8 Oct 2015 05:58:36 +0300 Subject: [PATCH 79/81] Add Single.fromCallable() --- src/main/java/rx/Single.java | 38 ++++++++++++++++++++++++++++++ src/test/java/rx/SingleTest.java | 40 ++++++++++++++++++++++++++++++++ 2 files changed, 78 insertions(+) diff --git a/src/main/java/rx/Single.java b/src/main/java/rx/Single.java index 4324d32acf..3701d93189 100644 --- a/src/main/java/rx/Single.java +++ b/src/main/java/rx/Single.java @@ -12,6 +12,7 @@ */ package rx; +import java.util.concurrent.Callable; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -605,6 +606,43 @@ public final static Single from(Future future, Scheduler sch return new Single(OnSubscribeToObservableFuture.toObservableFuture(future)).subscribeOn(scheduler); } + /** + * Returns a {@link Single} that invokes passed function and emits its result for each new Observer that subscribes. + *

+ * Allows you to defer execution of passed function until Observer subscribes to the {@link Single}. + * It makes passed function "lazy". + * Result of the function invocation will be emitted by the {@link Single}. + *

+ *
Scheduler:
+ *
{@code fromCallable} does not operate by default on a particular {@link Scheduler}.
+ *
+ * + * @param func + * function which execution should be deferred, it will be invoked when Observer will subscribe to the {@link Single}. + * @param + * the type of the item emitted by the {@link Single}. + * @return a {@link Single} whose {@link Observer}s' subscriptions trigger an invocation of the given function. + */ + @Experimental + public static Single fromCallable(final Callable func) { + return create(new OnSubscribe() { + @Override + public void call(SingleSubscriber singleSubscriber) { + final T value; + + try { + value = func.call(); + } catch (Throwable t) { + Exceptions.throwIfFatal(t); + singleSubscriber.onError(t); + return; + } + + singleSubscriber.onSuccess(value); + } + }); + } + /** * Returns a {@code Single} that emits a specified item. *

diff --git a/src/test/java/rx/SingleTest.java b/src/test/java/rx/SingleTest.java index 7d8fe2dc22..f78151b094 100644 --- a/src/test/java/rx/SingleTest.java +++ b/src/test/java/rx/SingleTest.java @@ -20,8 +20,10 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyZeroInteractions; +import static org.mockito.Mockito.when; import java.util.Arrays; +import java.util.concurrent.Callable; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -530,4 +532,42 @@ public void doOnErrorShouldThrowCompositeExceptionIfOnErrorActionThrows() { verify(action).call(error); } + + @Test + public void shouldEmitValueFromCallable() throws Exception { + Callable callable = mock(Callable.class); + + when(callable.call()).thenReturn("value"); + + TestSubscriber testSubscriber = new TestSubscriber(); + + Single + .fromCallable(callable) + .subscribe(testSubscriber); + + testSubscriber.assertValue("value"); + testSubscriber.assertNoErrors(); + + verify(callable).call(); + } + + @Test + public void shouldPassErrorFromCallable() throws Exception { + Callable callable = mock(Callable.class); + + Throwable error = new IllegalStateException(); + + when(callable.call()).thenThrow(error); + + TestSubscriber testSubscriber = new TestSubscriber(); + + Single + .fromCallable(callable) + .subscribe(testSubscriber); + + testSubscriber.assertNoValues(); + testSubscriber.assertError(error); + + verify(callable).call(); + } } From ffcd875c1ad88156c49115299690342c0f4fdf43 Mon Sep 17 00:00:00 2001 From: Nitesh Kant Date: Wed, 1 Jul 2015 18:25:08 -0700 Subject: [PATCH 80/81] New operators: `concatEmptyWith` and `mergeEmptyWith`. As discussed in issue #3037, the primary use of these operators is to be applied to `Observable` so that they can be merged and concatenated with an Observable of a different type. Both these operators raise an error if the source Observable emits any item. --- src/main/java/rx/Observable.java | 69 ++++++ .../operators/OperatorConcatEmptyWith.java | 203 ++++++++++++++++++ .../operators/OperatorMergeEmptyWith.java | 132 ++++++++++++ .../OperatorConcatEmptyWithTest.java | 167 ++++++++++++++ .../operators/OperatorMergeEmptyWithTest.java | 145 +++++++++++++ 5 files changed, 716 insertions(+) create mode 100644 src/main/java/rx/internal/operators/OperatorConcatEmptyWith.java create mode 100644 src/main/java/rx/internal/operators/OperatorMergeEmptyWith.java create mode 100644 src/test/java/rx/internal/operators/OperatorConcatEmptyWithTest.java create mode 100644 src/test/java/rx/internal/operators/OperatorMergeEmptyWithTest.java diff --git a/src/main/java/rx/Observable.java b/src/main/java/rx/Observable.java index 02142ec6ce..d8b863b499 100644 --- a/src/main/java/rx/Observable.java +++ b/src/main/java/rx/Observable.java @@ -9986,6 +9986,75 @@ public final Observable zipWith(Observable other, Func2 return zip(this, other, zipFunction); } + /** + * Returns an Observable that upon completion of the source Observable subscribes to the passed {@code other} + * Observable and then emits all items emitted by that Observable. This function does not expect the source + * Observable to emit any item, in case, the source Observable, emits any item, an {@link IllegalStateException} + * is raised. + *

+ * + * This is different than {@link #concatWith(Observable)} as it does not expect the source Observable to ever emit + * an item. So, this usually is useful for {@code Observable} and results in cleaner code as opposed to using + * a {@link #cast(Class)}, something like: + * + * {@code Observable.empty().cast(String.class).concatWith(Observable.just("Hello"))} + * + *

+ *
Scheduler:
+ *
{@code concatEmptyWith} does not operate by default on a particular {@link Scheduler}.
+ *
+ * + *
Backpressure:
+ *
{@code concatEmptyWith} does not propagate any demands from the subscriber to the source {@code Observable} + * as it never expects the source to ever emit an item. All demands are sent to the {@code other} + * {@code Observable}.
+ * + * @return an Observable that upon completion of the source, starts emitting items from the {@code other} + * Observable. + * @throws IllegalStateException If the source emits any item. + * + * @see #mergeEmptyWith(Observable) + */ + @Experimental + public final Observable concatEmptyWith(Observable other) { + return lift(new OperatorConcatEmptyWith(other)); + } + + /** + * Returns an Observable that only listens for error from the source Observable and emit items only from the passed + * {@code other} Observable. This function does not expect the source Observable to emit any item, in case, the + * source Observable, emits any item, an {@link IllegalStateException} is raised. + *

+ * + * This is different than {@link #mergeWith(Observable)} as it does not expect the source Observable to ever emit + * an item. So, this usually is useful for using on {@code Observable} and results in cleaner code as opposed + * to using a {@link #cast(Class)}, something like: + * {@code Observable.empty().cast(String.class).mergeWith(Observable.just("Hello"))} + * + *

+ *
Scheduler:
+ *
{@code mergeEmptyWith} does not operate by default on a particular {@link Scheduler}.
+ *
+ * + *
+ *
Backpressure:
+ *
{@code mergeEmptyWith} does not propagate any demands from the subscriber to the source {@code Observable} + * as it never expects the source to ever emit an item. All demands are sent to the {@code other} + * {@code Observable}.
+ *
+ * + * @return an Observable that only listens for errors from the source and starts emitting items from the + * {@code other} Observable on subscription. + * Observable. + * @throws IllegalStateException If the source emits any item. + * + * @see #concatEmptyWith(Observable) + */ + @Experimental + public final Observable mergeEmptyWith(Observable other) { + return lift(new OperatorMergeEmptyWith(other)); + } + /** * An Observable that never sends any information to an {@link Observer}. * This Observable is useful primarily for testing purposes. diff --git a/src/main/java/rx/internal/operators/OperatorConcatEmptyWith.java b/src/main/java/rx/internal/operators/OperatorConcatEmptyWith.java new file mode 100644 index 0000000000..6763ca16f2 --- /dev/null +++ b/src/main/java/rx/internal/operators/OperatorConcatEmptyWith.java @@ -0,0 +1,203 @@ +/** + * Copyright 2014 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.internal.operators; + +import rx.Observable; +import rx.Observable.Operator; +import rx.Producer; +import rx.Subscriber; +import rx.internal.producers.ProducerArbiter; +import rx.subscriptions.SerialSubscription; + +/** + * Returns an Observable that emits an error if any item is emitted by the source and emits items from the supplied + * alternate {@code Observable} after the source completes. + * + * @param the source value type + * @param the result value type + */ +public final class OperatorConcatEmptyWith implements Operator { + + private final Observable alternate; + + public OperatorConcatEmptyWith(Observable alternate) { + this.alternate = alternate; + } + + @Override + public Subscriber call(Subscriber child) { + final SerialSubscription ssub = new SerialSubscription(); + final ParentSubscriber parent = new ParentSubscriber(child, ssub, alternate); + ssub.set(parent); + child.add(ssub); + child.setProducer(parent.emptyProducer); + return parent; + } + + private final class ParentSubscriber extends Subscriber { + + private final Subscriber child; + private final SerialSubscription ssub; + private final EmptyProducer emptyProducer; + private final Observable alternate; + + ParentSubscriber(Subscriber child, final SerialSubscription ssub, Observable alternate) { + this.child = child; + this.ssub = ssub; + this.emptyProducer = new EmptyProducer(); + this.alternate = alternate; + } + + @Override + public void setProducer(final Producer producer) { + /* + * Always request Max from the parent as we never really expect the parent to emit an item, so the + * actual value does not matter. However, if the parent producer is waiting for a request to emit + * a terminal event, not requesting the same will cause a deadlock of the parent never completing and + * the child never subscribed. + */ + producer.request(Long.MAX_VALUE); + } + + @Override + public void onCompleted() { + if (!child.isUnsubscribed()) { + AlternateSubscriber as = new AlternateSubscriber(child, emptyProducer); + ssub.set(as); + alternate.unsafeSubscribe(as); + } + } + + @Override + public void onError(Throwable e) { + child.onError(e); + } + + @Override + public void onNext(T t) { + onError(new IllegalStateException("Concat empty with source emitted an item: " + t)); + } + } + + private final class AlternateSubscriber extends Subscriber { + + private final EmptyProducer emptyProducer; + private final Subscriber child; + + AlternateSubscriber(Subscriber child, EmptyProducer emptyProducer) { + this.child = child; + this.emptyProducer = emptyProducer; + } + + @Override + public void setProducer(final Producer producer) { + emptyProducer.setAltProducer(producer); + } + + @Override + public void onCompleted() { + child.onCompleted(); + } + + @Override + public void onError(Throwable e) { + child.onError(e); + } + + @Override + public void onNext(R r) { + child.onNext(r); + } + } + + /** + * This is a producer implementation that does the following: + * + *
    + *
  • If the alternate producer has not yet arrived, store the total requested count from downstream.
  • + *
  • If the alternate producer has arrived, then relay the request demand to it.
  • + *
  • Request {@link Long#MAX_VALUE} from the parent producer, the first time the child requests anything.
  • + *
+ * + * Since, this is only applicable to this operator, it does not check for emissions from the source, as the source + * is never expected to emit any item. Thus it is "lighter" weight than {@link ProducerArbiter} + */ + private static final class EmptyProducer implements Producer { + + /*Total requested items till the time the alternate producer arrives.*/ + private long missedRequested; /*Guarded by this*/ + /*Producer from the alternate Observable for this operator*/ + private Producer altProducer; /*Guarded by this*/ + + @Override + public void request(final long requested) { + if (requested < 0) { + throw new IllegalArgumentException("Requested items can not be negative."); + } + + if (requested == 0) { + return; + } + + boolean requestToAlternate = false; + Producer _altProducer; + synchronized (this) { + if (null == altProducer) { + /*Accumulate requested till the time an alternate producer arrives.*/ + long r = this.missedRequested; + long u = r + requested; + if (u < 0) { + u = Long.MAX_VALUE; + } + this.missedRequested = u; + } else { + /*If the alternate producer exists, then relay a valid request. The missed requested will be + requested from the alt producer on setProducer()*/ + requestToAlternate = true; + } + + _altProducer = altProducer; + } + + if (requestToAlternate) { + _altProducer.request(requested); + } + } + + private void setAltProducer(final Producer altProducer) { + if (null == altProducer) { + throw new IllegalArgumentException("Producer can not be null."); + } + + boolean requestToAlternate = false; + long _missedRequested; + + synchronized (this) { + if (0 != missedRequested) { + /*Something was requested from the source Observable, relay that to the new producer*/ + requestToAlternate = true; + } + + this.altProducer = altProducer; + _missedRequested = missedRequested; + } + + if (requestToAlternate) { + altProducer.request(_missedRequested); + } + } + } +} diff --git a/src/main/java/rx/internal/operators/OperatorMergeEmptyWith.java b/src/main/java/rx/internal/operators/OperatorMergeEmptyWith.java new file mode 100644 index 0000000000..21d15720c6 --- /dev/null +++ b/src/main/java/rx/internal/operators/OperatorMergeEmptyWith.java @@ -0,0 +1,132 @@ +/** + * Copyright 2014 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.internal.operators; + +import rx.Observable; +import rx.Observable.Operator; +import rx.Producer; +import rx.Subscriber; +import rx.observers.SerializedSubscriber; + +/** + * Returns an Observable that emits an error if any item is emitted by the source and emits items from the supplied + * alternate {@code Observable}. The errors from source are propagated as-is. + * + * @param the source value type + * @param the result value type + */ +public final class OperatorMergeEmptyWith implements Operator { + + private final Observable alternate; + + public OperatorMergeEmptyWith(Observable alternate) { + this.alternate = alternate; + } + + @Override + public Subscriber call(final Subscriber child) { + final ChildSubscriber wrappedChild = new ChildSubscriber(child); + final ParentSubscriber parent = new ParentSubscriber(wrappedChild); + wrappedChild.add(parent); + alternate.unsafeSubscribe(wrappedChild); + return parent; + } + + private final class ParentSubscriber extends Subscriber { + + private final ChildSubscriber child; + + ParentSubscriber(ChildSubscriber child) { + this.child = child; + } + + @Override + public void setProducer(final Producer producer) { + /* + * Always request Max from the parent as we never really expect the parent to emit an item, so the + * actual value does not matter. However, if the parent producer is waiting for a request to emit + * a terminal event, not requesting the same will cause the merged Observable to never complete. + */ + producer.request(Long.MAX_VALUE); + } + + @Override + public void onCompleted() { + child.parentCompleted(); + } + + @Override + public void onError(Throwable e) { + child.onError(e); + } + + @Override + public void onNext(T t) { + onError(new IllegalStateException("Merge empty with source emitted an item: " + t)); + } + } + + private final class ChildSubscriber extends Subscriber { + + private final SerializedSubscriber delegate; + private boolean parentCompleted; /*Guarded by this*/ + private boolean childCompleted; /*Guarded by this*/ + + ChildSubscriber(Subscriber delegate) { + super(delegate); + this.delegate = new SerializedSubscriber(delegate); + } + + @Override + public void onCompleted() { + boolean bothCompleted = false; + synchronized (this) { + if (parentCompleted) { + bothCompleted = true; + } + childCompleted = true; + } + + if (bothCompleted) { + delegate.onCompleted(); + } + } + + @Override + public void onError(Throwable e) { + delegate.onError(e); + } + + @Override + public void onNext(R r) { + delegate.onNext(r); + } + + public void parentCompleted() { + boolean bothCompleted = false; + synchronized (this) { + if (childCompleted) { + bothCompleted = true; + } + parentCompleted = true; + } + + if (bothCompleted) { + delegate.onCompleted(); + } + } + } +} diff --git a/src/test/java/rx/internal/operators/OperatorConcatEmptyWithTest.java b/src/test/java/rx/internal/operators/OperatorConcatEmptyWithTest.java new file mode 100644 index 0000000000..bf7bf35e32 --- /dev/null +++ b/src/test/java/rx/internal/operators/OperatorConcatEmptyWithTest.java @@ -0,0 +1,167 @@ +/** + * Copyright 2014 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.internal.operators; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import rx.Observable; +import rx.Observable.OnSubscribe; +import rx.Producer; +import rx.Subscriber; +import rx.observers.TestSubscriber; +import rx.schedulers.Schedulers; +import rx.schedulers.TestScheduler; + +public class OperatorConcatEmptyWithTest { + + @Rule + public final ExpectedException expectedException = ExpectedException.none(); + + @Test(timeout = 60000) + public void testWithVoid() { + final String soleValue = "Hello"; + Observable source = Observable.empty() + .concatEmptyWith(Observable.just(soleValue)); + + TestSubscriber testSubscriber = new TestSubscriber(); + source.subscribe(testSubscriber); + + testSubscriber.awaitTerminalEvent(); + testSubscriber.assertNoErrors(); + testSubscriber.assertValue(soleValue); + } + + @Test(timeout = 60000) + public void testErrorOnSourceEmitItem() { + Observable source = Observable.just(1) + .concatEmptyWith(Observable.just("Hello")); + + TestSubscriber testSubscriber = new TestSubscriber(); + source.subscribe(testSubscriber); + + testSubscriber.awaitTerminalEvent(); + testSubscriber.assertNoValues(); + testSubscriber.assertError(IllegalStateException.class); + } + + @Test(timeout = 60000) + public void testSourceError() throws Exception { + Observable source = Observable.error(new IllegalStateException()) + .concatEmptyWith(Observable.just("Hello")); + + TestSubscriber testSubscriber = new TestSubscriber(); + source.subscribe(testSubscriber); + + testSubscriber.awaitTerminalEvent(); + testSubscriber.assertNoValues(); + testSubscriber.assertError(IllegalStateException.class); + } + + @Test(timeout = 60000) + public void testNoSubscribeBeforeSourceCompletion() { + final String soleValue = "Hello"; + final TestScheduler testScheduler = Schedulers.test(); + + /*Delaying on complete event so to check that the subscription does not happen before completion*/ + Observable source = Observable.empty() + .observeOn(testScheduler) + .concatEmptyWith(Observable.just(soleValue)); + + TestSubscriber testSubscriber = new TestSubscriber(); + source.subscribe(testSubscriber); + + testSubscriber.assertNoTerminalEvent(); + testSubscriber.assertNoValues(); + + testScheduler.triggerActions(); + + testSubscriber.awaitTerminalEvent(); + testSubscriber.assertNoErrors(); + testSubscriber.assertValue(soleValue); + } + + @Test(timeout = 60000) + public void testRequestNSingle() throws Exception { + final String[] values = {"Hello1", "Hello2"}; + Observable source = Observable.empty() + .concatEmptyWith(Observable.from(values)); + + TestSubscriber testSubscriber = new TestSubscriber(0); + source.subscribe(testSubscriber); + + testSubscriber.assertNoTerminalEvent(); + testSubscriber.assertNoErrors(); + testSubscriber.assertNoValues(); + + testSubscriber.requestMore(2); + + testSubscriber.awaitTerminalEvent(); + testSubscriber.assertNoErrors(); + testSubscriber.assertValues(values); + } + + @Test(timeout = 60000) + public void testRequestNMulti() throws Exception { + final String[] values = {"Hello1", "Hello2"}; + Observable source = Observable.empty() + .concatEmptyWith(Observable.from(values)); + + TestSubscriber testSubscriber = new TestSubscriber(0); + source.subscribe(testSubscriber); + + testSubscriber.assertNoTerminalEvent(); + testSubscriber.assertNoErrors(); + testSubscriber.assertNoValues(); + + testSubscriber.requestMore(1); + + testSubscriber.assertNoTerminalEvent(); + testSubscriber.assertNoErrors(); + testSubscriber.assertValues(values[0]); + + testSubscriber.requestMore(1); + + testSubscriber.awaitTerminalEvent(); + testSubscriber.assertNoErrors(); + testSubscriber.assertValues(values); + } + + @Test(timeout = 60000) + public void testSourceDontCompleteWithoutRequest() throws Exception { + + TestSubscriber testSubscriber = new TestSubscriber(0); + + String soleValue = "Hello"; + Observable.create(new OnSubscribe() { + @Override + public void call(final Subscriber subscriber) { + subscriber.setProducer(new Producer() { + @Override + public void request(long n) { + subscriber.onCompleted(); + } + }); + } + }).concatEmptyWith(Observable.just(soleValue)).subscribe(testSubscriber); + + testSubscriber.requestMore(1); + + testSubscriber.awaitTerminalEvent(); + testSubscriber.assertNoErrors(); + testSubscriber.assertValue(soleValue); + } +} diff --git a/src/test/java/rx/internal/operators/OperatorMergeEmptyWithTest.java b/src/test/java/rx/internal/operators/OperatorMergeEmptyWithTest.java new file mode 100644 index 0000000000..0bc0ea6a1f --- /dev/null +++ b/src/test/java/rx/internal/operators/OperatorMergeEmptyWithTest.java @@ -0,0 +1,145 @@ +/** + * Copyright 2014 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.internal.operators; + +import org.junit.Test; +import rx.Observable; +import rx.Observable.OnSubscribe; +import rx.Producer; +import rx.Subscriber; +import rx.observers.TestSubscriber; +import rx.schedulers.Schedulers; +import rx.schedulers.TestScheduler; + +import java.util.concurrent.TimeUnit; + +public class OperatorMergeEmptyWithTest { + + @Test(timeout = 60000) + public void testWithVoid() { + final String soleValue = "Hello"; + Observable source = Observable.empty() + .mergeEmptyWith(Observable.just(soleValue)); + + TestSubscriber testSubscriber = new TestSubscriber(); + source.subscribe(testSubscriber); + + testSubscriber.awaitTerminalEvent(); + testSubscriber.assertNoErrors(); + testSubscriber.assertValue(soleValue); + } + + @Test(timeout = 60000) + public void testErrorOnSourceEmitItem() { + TestScheduler testScheduler = Schedulers.test(); + Observable source = Observable.just(1) + .mergeEmptyWith(Observable.just("Hello").observeOn(testScheduler)); + + TestSubscriber testSubscriber = new TestSubscriber(); + source.subscribe(testSubscriber); + + testScheduler.triggerActions(); + testSubscriber.awaitTerminalEvent(); + testSubscriber.assertNoValues(); + testSubscriber.assertError(IllegalStateException.class); + } + + @Test(timeout = 60000) + public void testSourceError() throws Exception { + TestScheduler testScheduler = Schedulers.test(); + Observable source = Observable.error(new IllegalStateException()) + .mergeEmptyWith(Observable.just("Hello").observeOn(testScheduler)); + + TestSubscriber testSubscriber = new TestSubscriber(); + source.subscribe(testSubscriber); + + testScheduler.triggerActions(); + testSubscriber.awaitTerminalEvent(); + testSubscriber.assertNoValues(); + testSubscriber.assertError(IllegalStateException.class); + } + + @Test(timeout = 60000) + public void testSourceComplete() throws Exception { + final String soleValue = "Hello"; + Observable source = Observable.empty() + .mergeEmptyWith(Observable.just(soleValue)); + + TestSubscriber testSubscriber = new TestSubscriber(); + source.subscribe(testSubscriber); + + testSubscriber.awaitTerminalEvent(); + testSubscriber.assertNoErrors(); + testSubscriber.assertValue(soleValue); + } + + @Test(timeout = 60000) + public void testErrorFromSourcePostEmission() { + final String soleValue = "Hello"; + final TestScheduler testScheduler = Schedulers.test(); + + /*Delaying error event*/ + Observable source = Observable.error(new IllegalArgumentException()) + .observeOn(testScheduler) + .mergeEmptyWith(Observable.just(soleValue)); + + TestSubscriber testSubscriber = new TestSubscriber(); + source.subscribe(testSubscriber); + + testSubscriber.assertNotCompleted(); + testSubscriber.assertNoErrors(); + testSubscriber.assertValue(soleValue); + + testScheduler.advanceTimeBy(1, TimeUnit.HOURS); + testSubscriber.awaitTerminalEvent(); + testSubscriber.assertError(IllegalArgumentException.class); + } + + @Test(timeout = 60000) + public void testSourceNeverCompletes() throws Exception { + TestSubscriber subscriber = new TestSubscriber(); + Observable.never() + .mergeEmptyWith(Observable.just("Hello")) + .subscribe(subscriber); + + subscriber.assertValue("Hello"); + subscriber.assertNoTerminalEvent(); + } + + @Test(timeout = 60000) + public void testSourceDoesntCompleteWithoutRequest() throws Exception { + TestSubscriber testSubscriber = new TestSubscriber(0); + + String soleValue = "Hello"; + Observable.create(new OnSubscribe() { + @Override + public void call(final Subscriber subscriber) { + subscriber.setProducer(new Producer() { + @Override + public void request(long n) { + subscriber.onCompleted(); + } + }); + } + }).mergeEmptyWith(Observable.just(soleValue)).subscribe(testSubscriber); + + testSubscriber.requestMore(1); + + testSubscriber.awaitTerminalEvent(); + testSubscriber.assertNoErrors(); + testSubscriber.assertValue(soleValue); + } +} From 22f6ba7f4a9d59b8917012ec88976b35fac672c4 Mon Sep 17 00:00:00 2001 From: Artem Zinnatullin Date: Thu, 8 Oct 2015 05:40:12 +0300 Subject: [PATCH 81/81] Add Single.doOnSuccess() --- src/main/java/rx/Single.java | 34 ++++++++++++++ src/test/java/rx/SingleTest.java | 78 ++++++++++++++++++++++++++++++++ 2 files changed, 112 insertions(+) diff --git a/src/main/java/rx/Single.java b/src/main/java/rx/Single.java index 3701d93189..6817a4e283 100644 --- a/src/main/java/rx/Single.java +++ b/src/main/java/rx/Single.java @@ -1864,4 +1864,38 @@ public void onNext(T t) { return lift(new OperatorDoOnEach(observer)); } + + /** + * Modifies the source {@link Single} so that it invokes an action when it calls {@code onSuccess}. + *

+ * + *

+ *
Scheduler:
+ *
{@code doOnSuccess} does not operate by default on a particular {@link Scheduler}.
+ *
+ * + * @param onSuccess + * the action to invoke when the source {@link Single} calls {@code onSuccess} + * @return the source {@link Single} with the side-effecting behavior applied + * @see ReactiveX operators documentation: Do + */ + @Experimental + public final Single doOnSuccess(final Action1 onSuccess) { + Observer observer = new Observer() { + @Override + public void onCompleted() { + } + + @Override + public void onError(Throwable e) { + } + + @Override + public void onNext(T t) { + onSuccess.call(t); + } + }; + + return lift(new OperatorDoOnEach(observer)); + } } diff --git a/src/test/java/rx/SingleTest.java b/src/test/java/rx/SingleTest.java index f78151b094..de1a38f0ca 100644 --- a/src/test/java/rx/SingleTest.java +++ b/src/test/java/rx/SingleTest.java @@ -16,6 +16,7 @@ import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.mockito.Matchers.eq; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; @@ -570,4 +571,81 @@ public void shouldPassErrorFromCallable() throws Exception { verify(callable).call(); } + + @Test + public void doOnSuccessShouldInvokeAction() { + Action1 action = mock(Action1.class); + + TestSubscriber testSubscriber = new TestSubscriber(); + + Single + .just("value") + .doOnSuccess(action) + .subscribe(testSubscriber); + + testSubscriber.assertValue("value"); + testSubscriber.assertNoErrors(); + + verify(action).call(eq("value")); + } + + @Test + public void doOnSuccessShouldPassErrorFromActionToSubscriber() { + Action1 action = mock(Action1.class); + + Throwable error = new IllegalStateException(); + doThrow(error).when(action).call(eq("value")); + + TestSubscriber testSubscriber = new TestSubscriber(); + + Single + .just("value") + .doOnSuccess(action) + .subscribe(testSubscriber); + + testSubscriber.assertNoValues(); + testSubscriber.assertError(error); + + verify(action).call(eq("value")); + } + + @Test + public void doOnSuccessShouldNotCallActionIfSingleThrowsError() { + Action1 action = mock(Action1.class); + + Throwable error = new IllegalStateException(); + + TestSubscriber testSubscriber = new TestSubscriber(); + + Single + .error(error) + .doOnSuccess(action) + .subscribe(testSubscriber); + + testSubscriber.assertNoValues(); + testSubscriber.assertError(error); + + verifyZeroInteractions(action); + } + + @Test + public void doOnSuccessShouldNotSwallowExceptionThrownByAction() { + Action1 action = mock(Action1.class); + + Throwable exceptionFromAction = new IllegalStateException(); + + doThrow(exceptionFromAction).when(action).call(eq("value")); + + TestSubscriber testSubscriber = new TestSubscriber(); + + Single + .just("value") + .doOnSuccess(action) + .subscribe(testSubscriber); + + testSubscriber.assertNoValues(); + testSubscriber.assertError(exceptionFromAction); + + verify(action).call(eq("value")); + } }